Skip to content

Commit f2688ef

Browse files
xysunzsxwing
authored andcommitted
[SPARK-21996][SQL] read files with space in name for streaming
## What changes were proposed in this pull request? Structured streaming is now able to read files with space in file name (previously it would skip the file and output a warning) ## How was this patch tested? Added new unit test. Author: Xiayun Sun <[email protected]> Closes #19247 from xysun/SPARK-21996. (cherry picked from commit 0219470) Signed-off-by: Shixiong Zhu <[email protected]>
1 parent 050c1e2 commit f2688ef

File tree

2 files changed

+49
-3
lines changed

2 files changed

+49
-3
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -166,7 +166,7 @@ class FileStreamSource(
166166
val newDataSource =
167167
DataSource(
168168
sparkSession,
169-
paths = files.map(_.path),
169+
paths = files.map(f => new Path(new URI(f.path)).toString),
170170
userSpecifiedSchema = Some(schema),
171171
partitionColumns = partitionColumns,
172172
className = fileFormatClassName,

sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala

Lines changed: 48 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -74,11 +74,11 @@ abstract class FileStreamSourceTest
7474
protected def addData(source: FileStreamSource): Unit
7575
}
7676

77-
case class AddTextFileData(content: String, src: File, tmp: File)
77+
case class AddTextFileData(content: String, src: File, tmp: File, tmpFilePrefix: String = "text")
7878
extends AddFileData {
7979

8080
override def addData(source: FileStreamSource): Unit = {
81-
val tempFile = Utils.tempFileWith(new File(tmp, "text"))
81+
val tempFile = Utils.tempFileWith(new File(tmp, tmpFilePrefix))
8282
val finalFile = new File(src, tempFile.getName)
8383
src.mkdirs()
8484
require(stringToFile(tempFile, content).renameTo(finalFile))
@@ -408,6 +408,52 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
408408
}
409409
}
410410

411+
test("SPARK-21996 read from text files -- file name has space") {
412+
withTempDirs { case (src, tmp) =>
413+
val textStream = createFileStream("text", src.getCanonicalPath)
414+
val filtered = textStream.filter($"value" contains "keep")
415+
416+
testStream(filtered)(
417+
AddTextFileData("drop1\nkeep2\nkeep3", src, tmp, "text text"),
418+
CheckAnswer("keep2", "keep3")
419+
)
420+
}
421+
}
422+
423+
test("SPARK-21996 read from text files generated by file sink -- file name has space") {
424+
val testTableName = "FileStreamSourceTest"
425+
withTable(testTableName) {
426+
withTempDirs { case (src, checkpoint) =>
427+
val output = new File(src, "text text")
428+
val inputData = MemoryStream[String]
429+
val ds = inputData.toDS()
430+
431+
val query = ds.writeStream
432+
.option("checkpointLocation", checkpoint.getCanonicalPath)
433+
.format("text")
434+
.start(output.getCanonicalPath)
435+
436+
try {
437+
inputData.addData("foo")
438+
failAfter(streamingTimeout) {
439+
query.processAllAvailable()
440+
}
441+
} finally {
442+
query.stop()
443+
}
444+
445+
val df2 = spark.readStream.format("text").load(output.getCanonicalPath)
446+
val query2 = df2.writeStream.format("memory").queryName(testTableName).start()
447+
try {
448+
query2.processAllAvailable()
449+
checkDatasetUnorderly(spark.table(testTableName).as[String], "foo")
450+
} finally {
451+
query2.stop()
452+
}
453+
}
454+
}
455+
}
456+
411457
test("read from textfile") {
412458
withTempDirs { case (src, tmp) =>
413459
val textStream = spark.readStream.textFile(src.getCanonicalPath)

0 commit comments

Comments
 (0)