Github user zsxwing commented on a diff in the pull request:
https://github.com/apache/spark/pull/11034#discussion_r52209936
--- Diff:
sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
---
@@ -0,0 +1,319 @@
+/*
+ * 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.{ByteArrayInputStream, File, InputStream}
+
+import com.google.common.base.Charsets.UTF_8
+
+import org.apache.spark.sql.StreamTest
+import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.FileStreamSource._
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.{StringType, StructType}
+import org.apache.spark.util.Utils
+
+class FileStreamSourceTest extends StreamTest with SharedSQLContext {
+
+ import testImplicits._
+
+ case class AddTextFileData(source: FileStreamSource, content: String,
src: File, tmp: File)
+ extends AddData {
+
+ override def addData(): Offset = {
+ source.currentOffset {
+ val file = Utils.tempFileWith(new File(tmp, "text"))
+ stringToFile(file, content).renameTo(new File(src, file.getName))
+ } + 1
+ }
+ }
+
+ case class AddParquetFileData(
+ source: FileStreamSource,
+ content: Seq[String],
+ src: File,
+ tmp: File) extends AddData {
+
+ override def addData(): Offset = {
+ source.currentOffset {
+ val file = Utils.tempFileWith(new File(tmp, "parquet"))
+ content.toDS().toDF().write.parquet(file.getCanonicalPath)
+ file.renameTo(new File(src, file.getName))
+ } + 1
+ }
+ }
+
+ /** Use `format` and `path` to create FileStreamSource via
DataFrameReader */
+ def createFileStreamSource(
+ format: String,
+ path: String,
+ schema: Option[StructType] = None): FileStreamSource = {
+ val reader =
+ if (schema.isDefined) {
+ sqlContext.read.format(format).schema(schema.get)
+ } else {
+ sqlContext.read.format(format)
+ }
+ reader.stream(path)
+ .queryExecution.analyzed
+ .collect { case StreamingRelation(s: FileStreamSource, _) => s }
+ .head
+ }
+
+ val valueSchema = new StructType().add("value", StringType)
+}
+
+class FileStreamSourceSuite extends FileStreamSourceTest with
SharedSQLContext {
+
+ import testImplicits._
+
+ test("read from text files") {
+ val src = Utils.createTempDir("streaming.src")
+ val tmp = Utils.createTempDir("streaming.tmp")
+
+ val textSource = createFileStreamSource("text", src.getCanonicalPath)
+ val df = textSource.toDF().filter($"value" contains "keep")
+ val filtered = df
+
+ testStream(filtered)(
+ AddTextFileData(textSource, "drop1\nkeep2\nkeep3", src, tmp),
+ CheckAnswer("keep2", "keep3"),
+ StopStream,
+ AddTextFileData(textSource, "drop4\nkeep5\nkeep6", src, tmp),
+ StartStream,
+ 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 = createFileStreamSource("json", src.getCanonicalPath,
Some(valueSchema))
+ val df = textSource.toDF().filter($"value" contains "keep")
+ val filtered = df
+
+ testStream(filtered)(
+ AddTextFileData(
+ textSource,
+ "{'value': 'drop1'}\n{'value': 'keep2'}\n{'value': 'keep3'}",
+ src,
+ tmp),
+ CheckAnswer("keep2", "keep3"),
+ StopStream,
+ AddTextFileData(
+ textSource,
+ "{'value': 'drop4'}\n{'value': 'keep5'}\n{'value': 'keep6'}",
+ src,
+ tmp),
+ StartStream,
+ CheckAnswer("keep2", "keep3", "keep5", "keep6"),
+ AddTextFileData(
+ textSource,
+ "{'value': 'drop7'}\n{'value': 'keep8'}\n{'value': 'keep9'}",
+ src,
+ tmp),
+ CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9")
+ )
+
+ Utils.deleteRecursively(src)
+ Utils.deleteRecursively(tmp)
+ }
+
+ test("read from json files with inferring schema") {
+ val src = Utils.createTempDir("streaming.src")
+ val tmp = Utils.createTempDir("streaming.tmp")
+
+ // Add a file so that we can infer its schema
+ stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c':
'keep2'}\n{'c': 'keep3'}")
+
+ val textSource = createFileStreamSource("json", src.getCanonicalPath)
+
+ // FileStreamSource should infer the column "c"
+ val df = textSource.toDF().filter($"c" contains "keep")
+ val filtered = df
+
+ testStream(filtered)(
+ AddTextFileData(textSource, "{'c': 'drop4'}\n{'c': 'keep5'}\n{'c':
'keep6'}", src, tmp),
+ CheckAnswer("keep2", "keep3", "keep5", "keep6")
+ )
+
+ 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 = createFileStreamSource("parquet",
src.getCanonicalPath, Some(valueSchema))
+ val df = fileSource.toDF().filter($"value" contains "keep")
+ val filtered = df
+
+ testStream(filtered)(
+ AddParquetFileData(fileSource, Seq("drop1", "keep2", "keep3"), src,
tmp),
+ CheckAnswer("keep2", "keep3"),
+ StopStream,
+ AddParquetFileData(fileSource, Seq("drop4", "keep5", "keep6"), src,
tmp),
+ StartStream,
+ 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)
+ }
+
+ test("file stream source without schema") {
+ val src = Utils.createTempDir("streaming.src")
+
+ // Only "text" doesn't need a schema
+ createFileStreamSource("text", src.getCanonicalPath)
--- End diff --
Fixed the path error for `stream`
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]