-
Notifications
You must be signed in to change notification settings - Fork 29k
[Spark-21996][SQL] read files with space in name for streaming #19247
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 4 commits
38901ef
4f5979a
435f42b
2542014
f07ccbe
7342d6c
6e4f809
04c2b14
10106b3
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 |
|---|---|---|
|
|
@@ -86,6 +86,18 @@ abstract class FileStreamSourceTest | |
| } | ||
| } | ||
|
|
||
| case class AddTextFileDataWithSpaceInFileName(content: String, src: File, tmp: File) | ||
| extends AddFileData { | ||
|
|
||
| override def addData(source: FileStreamSource): Unit = { | ||
| val tempFile = Utils.tempFileWith(new File(tmp, "text text")) | ||
| val finalFile = new File(src, tempFile.getName) | ||
| src.mkdirs() | ||
| require(stringToFile(tempFile, content).renameTo(finalFile)) | ||
| logInfo(s"Written text '$content' to file $finalFile") | ||
| } | ||
| } | ||
|
|
||
| case class AddOrcFileData(data: DataFrame, src: File, tmp: File) extends AddFileData { | ||
| override def addData(source: FileStreamSource): Unit = { | ||
| AddOrcFileData.writeToFile(data, src, tmp) | ||
|
|
@@ -408,6 +420,18 @@ class FileStreamSourceSuite extends FileStreamSourceTest { | |
| } | ||
| } | ||
|
|
||
| test("SPARK-21996 read from text files -- file name has space") { | ||
|
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 we run the same test also for the other input format, ie. parquet, orc, ... ?
Member
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 test should be enough. The issue is in file stream source.
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. yes, for this PR it is, but it would be great if we can ensure that all the data sources have the same behavior... Maybe we can do this is another PR if you think it is better
Member
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. Not. I meant it's not an issue of file formats. There are not some special codes in file stream source. If there should be any tests for such issue, they should be inside file format tests. |
||
| withTempDirs { case (src, tmp) => | ||
| val textStream = createFileStream("text", src.getCanonicalPath) | ||
| val filtered = textStream.filter($"value" contains "keep") | ||
|
|
||
| testStream(filtered)( | ||
| AddTextFileDataWithSpaceInFileName("drop1\nkeep2\nkeep3", src, tmp), | ||
| CheckAnswer("keep2", "keep3") | ||
| ) | ||
| } | ||
| } | ||
|
|
||
| test("read from textfile") { | ||
| withTempDirs { case (src, tmp) => | ||
| val textStream = spark.readStream.textFile(src.getCanonicalPath) | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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 would suggest that adding a new parameter to AddTextFileData rather than introducing a new class, such as
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.
sure will update and add the test for file sink. Thanks for the review.