-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19633][SS] FileSource read from FileSink #16987
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
Closed
Closed
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -52,10 +52,7 @@ abstract class FileStreamSourceTest | |
| query.nonEmpty, | ||
| "Cannot add data when there is no query for finding the active file stream source") | ||
|
|
||
| val sources = query.get.logicalPlan.collect { | ||
| case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] => | ||
| source.asInstanceOf[FileStreamSource] | ||
| } | ||
| val sources = getSourcesFromStreamingQuery(query.get) | ||
| if (sources.isEmpty) { | ||
| throw new Exception( | ||
| "Could not find file source in the StreamExecution logical plan to add data 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. this common logic is extracted out |
||
|
|
@@ -134,6 +131,14 @@ abstract class FileStreamSourceTest | |
| }.head | ||
| } | ||
|
|
||
| protected def getSourcesFromStreamingQuery(query: StreamExecution): Seq[FileStreamSource] = { | ||
| query.logicalPlan.collect { | ||
| case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] => | ||
| source.asInstanceOf[FileStreamSource] | ||
| } | ||
| } | ||
|
|
||
|
|
||
| protected def withTempDirs(body: (File, File) => Unit) { | ||
| val src = Utils.createTempDir(namePrefix = "streaming.src") | ||
| val tmp = Utils.createTempDir(namePrefix = "streaming.tmp") | ||
|
|
@@ -388,9 +393,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { | |
| CheckAnswer("a", "b", "c", "d"), | ||
|
|
||
| AssertOnQuery("seen files should contain only one entry") { streamExecution => | ||
| val source = streamExecution.logicalPlan.collect { case e: StreamingExecutionRelation => | ||
| e.source.asInstanceOf[FileStreamSource] | ||
| }.head | ||
| val source = getSourcesFromStreamingQuery(streamExecution).head | ||
| assert(source.seenFiles.size == 1) | ||
| true | ||
| } | ||
|
|
@@ -662,6 +665,101 @@ class FileStreamSourceSuite extends FileStreamSourceTest { | |
| } | ||
| } | ||
|
|
||
| test("read data from outputs of another streaming query") { | ||
| withSQLConf(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3") { | ||
| withTempDirs { case (outputDir, checkpointDir) => | ||
| // q1 is a streaming query that reads from memory and writes to text files | ||
| val q1Source = MemoryStream[String] | ||
| val q1 = | ||
| q1Source | ||
| .toDF() | ||
| .writeStream | ||
| .option("checkpointLocation", checkpointDir.getCanonicalPath) | ||
| .format("text") | ||
| .start(outputDir.getCanonicalPath) | ||
|
|
||
| // q2 is a streaming query that reads q1's text outputs | ||
| val q2 = | ||
| createFileStream("text", outputDir.getCanonicalPath).filter($"value" contains "keep") | ||
|
|
||
| def q1AddData(data: String*): StreamAction = | ||
| Execute { _ => | ||
| q1Source.addData(data) | ||
| q1.processAllAvailable() | ||
| } | ||
| def q2ProcessAllAvailable(): StreamAction = Execute { q2 => q2.processAllAvailable() } | ||
|
|
||
| testStream(q2)( | ||
| // batch 0 | ||
| q1AddData("drop1", "keep2"), | ||
| q2ProcessAllAvailable(), | ||
| CheckAnswer("keep2"), | ||
|
|
||
| // batch 1 | ||
| Assert { | ||
| // create a text file that won't be on q1's sink log | ||
| // thus even if its content contains "keep", it should NOT appear in q2's answer | ||
| val shouldNotKeep = new File(outputDir, "should_not_keep.txt") | ||
| stringToFile(shouldNotKeep, "should_not_keep!!!") | ||
| shouldNotKeep.exists() | ||
| }, | ||
| q1AddData("keep3"), | ||
| q2ProcessAllAvailable(), | ||
| CheckAnswer("keep2", "keep3"), | ||
|
|
||
| // batch 2: check that things work well when the sink log gets compacted | ||
| q1AddData("keep4"), | ||
| Assert { | ||
| // compact interval is 3, so file "2.compact" should exist | ||
| new File(outputDir, s"${FileStreamSink.metadataDir}/2.compact").exists() | ||
| }, | ||
| q2ProcessAllAvailable(), | ||
| CheckAnswer("keep2", "keep3", "keep4"), | ||
|
|
||
| Execute { _ => q1.stop() } | ||
| ) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("start before another streaming query, and read its output") { | ||
| withTempDirs { case (outputDir, checkpointDir) => | ||
| // q1 is a streaming query that reads from memory and writes to text files | ||
| val q1Source = MemoryStream[String] | ||
| // define q1, but don't start it for now | ||
| val q1Write = | ||
| q1Source | ||
| .toDF() | ||
| .writeStream | ||
| .option("checkpointLocation", checkpointDir.getCanonicalPath) | ||
| .format("text") | ||
| var q1: StreamingQuery = null | ||
|
|
||
| val q2 = createFileStream("text", outputDir.getCanonicalPath).filter($"value" contains "keep") | ||
|
|
||
| testStream(q2)( | ||
| AssertOnQuery { q2 => | ||
| val fileSource = getSourcesFromStreamingQuery(q2).head | ||
| // q1 has not started yet, verify that q2 doesn't know whether q1 has metadata | ||
| fileSource.sourceHasMetadata === None | ||
| }, | ||
| Execute { _ => | ||
| q1 = q1Write.start(outputDir.getCanonicalPath) | ||
| q1Source.addData("drop1", "keep2") | ||
| q1.processAllAvailable() | ||
| }, | ||
| AssertOnQuery { q2 => | ||
| q2.processAllAvailable() | ||
| val fileSource = getSourcesFromStreamingQuery(q2).head | ||
| // q1 has started, verify that q2 knows q1 has metadata by now | ||
| fileSource.sourceHasMetadata === Some(true) | ||
| }, | ||
| CheckAnswer("keep2"), | ||
| Execute { _ => q1.stop() } | ||
| ) | ||
| } | ||
| } | ||
|
|
||
| test("when schema inference is turned on, should read partition data") { | ||
| def createFile(content: String, src: File, tmp: File): Unit = { | ||
| val tempFile = Utils.tempFileWith(new File(tmp, "text")) | ||
|
|
@@ -755,10 +853,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { | |
| .streamingQuery | ||
| q.processAllAvailable() | ||
| val memorySink = q.sink.asInstanceOf[MemorySink] | ||
| val fileSource = q.logicalPlan.collect { | ||
| case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] => | ||
| source.asInstanceOf[FileStreamSource] | ||
| }.head | ||
| val fileSource = getSourcesFromStreamingQuery(q).head | ||
|
|
||
| /** Check the data read in the last batch */ | ||
| def checkLastBatchData(data: Int*): Unit = { | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
( some notes here since the changes are not trival )
here we're using this
sourceHasMetadatato indicate whether we know for sure the source has metadata, as stated in the source file comments:Nonemeans we don't know at the momentSome(true)means we know for sure the source DOES have metadataSome(false)means we know for sure the source DOSE NOT have metadata