Skip to content
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

Closed
wants to merge 16 commits into from
Closed

[SPARK-13149][SQL]Add FileStreamSource #11034

wants to merge 16 commits into from

Conversation

zsxwing
Copy link
Member

@zsxwing zsxwing commented Feb 2, 2016

FileStreamSource is an implementation of org.apache.spark.sql.execution.streaming.Source. It takes advantage of the existing HadoopFsRelationProvider to support various file formats. It remembers files in each batch and stores it into the metadata files so as to recover them when restarting. The metadata files are stored in the file system. There will be a further PR to clean up the metadata files periodically.

This is based on the initial work from @marmbrus.

@zsxwing zsxwing changed the title Add FileStreamSource and a simple version of FileStreamSink [SPARK-13149][SQL]Add FileStreamSource and a simple version of FileStreamSink Feb 2, 2016
@zsxwing
Copy link
Member Author

zsxwing commented Feb 2, 2016

@marmbrus @tdas Please take a look.

@SparkQA
Copy link

SparkQA commented Feb 3, 2016

Test build #50604 has finished for PR 11034 at commit a2784ff.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class FileStreamSink(
    • class FileStreamSource(
    • trait HadoopFsRelationProvider extends StreamSourceProvider with StreamSinkProvider

import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.util.Utils

class FileStreamSourceSuite extends StreamTest with SharedSQLContext {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we write these tests with the MemorySink instead, using testStream? I'd like to be able to test things like dropped batches and restarting as well. I would also be good to have them less coupled.

@zsxwing zsxwing changed the title [SPARK-13149][SQL]Add FileStreamSource and a simple version of FileStreamSink [SPARK-13149][SQL]Add FileStreamSource Feb 3, 2016
@SparkQA
Copy link

SparkQA commented Feb 3, 2016

Test build #50678 has finished for PR 11034 at commit 71e6312.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Member Author

zsxwing commented Feb 3, 2016

retest this please

@SparkQA
Copy link

SparkQA commented Feb 3, 2016

Test build #50684 has finished for PR 11034 at commit 6a90c55.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 3, 2016

Test build #50687 has finished for PR 11034 at commit 6a90c55.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

import sqlContext.implicits._

/** Returns the schema of the data from this source */
override def schema: StructType = dataSchema.getOrElse(new StructType().add("value", StringType))
Copy link
Contributor

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 using dataFrameBuilder and extract the schema from there.

We should also add tests that would catch a problem here.

Copy link
Member Author

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.

dataSchema.getOrElse {
val filesPresent = fetchAllFiles()
if (filesPresent.isEmpty) {
new StructType().add("value", StringType)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Even if there are no files present, we should probably still defer to the source. Those that can support that will work and those that don't will throw the correct error message.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Those that can support that will work and those that don't will throw the correct error message.

But we need to return some StructType here. Any magic to defer that?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh I see, even sqlContext.read.format("text").load() doesn't work? I would rather fix that than hardcode this here.

For sources like parquet/json it doesn't really make sense to let them point it at an empty directory so I would rather throw an error.

@SparkQA
Copy link

SparkQA commented Feb 5, 2016

Test build #50779 has finished for PR 11034 at commit 93af82e.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 5, 2016

Test build #50780 has finished for PR 11034 at commit 2af6fc8.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 5, 2016

Test build #50830 has finished for PR 11034 at commit ce0556d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 5, 2016

Test build #50838 has finished for PR 11034 at commit 9a1042c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

* an empty `Seq`.
*/
def readBatch(input: InputStream): Seq[String] = {
val lines = scala.io.Source.fromInputStream(input)(Codec.UTF8).getLines().toArray
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should validate the version too probably?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@marmbrus
Copy link
Contributor

marmbrus commented Feb 5, 2016

Small comments, otherwise LGTM

@SparkQA
Copy link

SparkQA commented Feb 6, 2016

Test build #50848 has finished for PR 11034 at commit 1ffee5f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class Result(avgMs: Double, bestRate: Double, bestMs: Double)
    • s\"Unable to generate an encoder for inner class$`
    • case class NaturalJoin(tpe: JoinType) extends JoinType

val src = Utils.createTempDir("streaming.src")

// Only "text" doesn't need a schema
createFileStreamSource("text", src.getCanonicalPath)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we also make sure we throw a better error for this case?

scala> sqlContext.read.format("text").stream()
java.util.NoSuchElementException: key not found: path
  at scala.collection.MapLike$class.default(MapLike.scala:228)
  at scala.collection.AbstractMap.default(Map.scala:59)
  at scala.collection.MapLike$class.apply(MapLike.scala:141)
  at scala.collection.AbstractMap.apply(Map.scala:59)
  at org.apache.spark.sql.sources.HadoopFsRelationProvider$class.createSource(interfaces.scala:206)
  at org.apache.spark.sql.execution.datasources.text.DefaultSource.createSource(DefaultSource.scala:42)
  at org.apache.spark.sql.execution.datasources.ResolvedDataSource$.createSource(ResolvedDataSource.scala:107)
  at org.apache.spark.sql.DataFrameReader.stream(DataFrameReader.scala:167)
  ... 40 elided

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would like to fix it in a separate PR since load throws the same error:

scala> sqlContext.read.format("text").load()
java.util.NoSuchElementException: key not found: path
  at scala.collection.MapLike$class.default(MapLike.scala:228)
  at org.apache.spark.sql.execution.datasources.CaseInsensitiveMap.default(ddl.scala:159)
  at scala.collection.MapLike$class.apply(MapLike.scala:141)
  at org.apache.spark.sql.execution.datasources.CaseInsensitiveMap.apply(ddl.scala:159)
  at org.apache.spark.sql.execution.datasources.ResolvedDataSource$$anonfun$10.apply(ResolvedDataSource.scala:200)
  at org.apache.spark.sql.execution.datasources.ResolvedDataSource$$anonfun$10.apply(ResolvedDataSource.scala:200)
  at scala.Option.getOrElse(Option.scala:121)
  at org.apache.spark.sql.execution.datasources.ResolvedDataSource$.apply(ResolvedDataSource.scala:200)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:129)
  ... 49 elided

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed the path error for stream

@zsxwing
Copy link
Member Author

zsxwing commented Feb 8, 2016

retest this please

@SparkQA
Copy link

SparkQA commented Feb 8, 2016

Test build #50937 has finished for PR 11034 at commit fb0e3f9.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

private val batchToMetadata = new HashMap[Long, Seq[String]]

{
// Restore statues from the metadata files
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: statuses, not statues. :) Also, what is status? Isnt it just file names?

@tdas
Copy link
Contributor

tdas commented Feb 9, 2016

Offline discussion:
It will be good to have a unit test that documents/tests the behavior of what file source gets generated when format and/or schema is provided/not provided and existing files present / not present, what is the schema generated in the source. So that we have clear understanding of what the behavior is.

@SparkQA
Copy link

SparkQA commented Feb 10, 2016

Test build #51009 has finished for PR 11034 at commit 07e2ddd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@tdas
Copy link
Contributor

tdas commented Feb 10, 2016

LGTM. New tests are great, and they make sense. Merging this!
Thanks @zsxwing and @marmbrus

@asfgit asfgit closed this in b385ce3 Feb 10, 2016
@zsxwing zsxwing deleted the stream-df-file-source branch February 10, 2016 18:59
asfgit pushed a commit that referenced this pull request Feb 21, 2016
Improved the error message as per discussion in #11034 (comment). Also made `path` and `metadataPath` in FileStreamSource case insensitive.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11154 from zsxwing/path.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants