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
Show file tree
Hide file tree
Changes from 12 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ object ResolvedDataSource extends Logging {
s"Data source $providerName does not support streamed reading")
}

provider.createSource(sqlContext, options, userSpecifiedSchema)
provider.createSource(sqlContext, userSpecifiedSchema, providerName, options)
}

def createSink(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,229 @@
/*
* 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.execution.streaming

import java.io._

import scala.collection.mutable.{ArrayBuffer, HashMap}
import scala.io.Codec

import com.google.common.base.Charsets.UTF_8
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}

import org.apache.spark.Logging
import org.apache.spark.sql.{DataFrame, SQLContext}
import org.apache.spark.sql.types.{StringType, StructType}
import org.apache.spark.util.collection.OpenHashSet

/**
* A very simple source that reads text files from the given directory as they appear.
*
* TODO Clean up the metadata files periodically
*/
class FileStreamSource(
sqlContext: SQLContext,
metadataPath: String,
path: String,
dataSchema: Option[StructType],
providerName: String,
dataFrameBuilder: Array[String] => DataFrame) extends Source with Logging {

private val fs = FileSystem.get(sqlContext.sparkContext.hadoopConfiguration)
private var maxBatchId = -1
private val seenFiles = new OpenHashSet[String]

/** Cache files for each batch. The content of this map is also stored in the disk. */
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: Just say, Map of batch id to files. This map is also stored in the source directory.

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?

val existingBatchFiles = fetchAllBatchFiles()
if (existingBatchFiles.nonEmpty) {
val existingBatchIds = existingBatchFiles.map(_.getPath.getName.toInt)
maxBatchId = existingBatchIds.max
// Recover "batchToMetadata" and "seenFiles" from existing metadata files.
existingBatchIds.sorted.foreach { batchId =>
val files = readBatch(batchId)
if (files.isEmpty) {
// Assert that the corrupted file must be the latest metadata file.
require(batchId == maxBatchId, "Invalid 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.

this should be a illegalstateexception, not illegalargumentexception since this is not based on some argument that user did.

maxBatchId = maxBatchId - 1
} else {
batchToMetadata(batchId) = files
files.foreach(seenFiles.add)
}
}
}
}

/** Returns the schema of the data from this source */
override lazy val schema: StructType = {
dataSchema.getOrElse {
val filesPresent = fetchAllFiles()
if (filesPresent.isEmpty) {
if (providerName == "text") {
// Add a default schema for "text"
new StructType().add("value", StringType)
} else {
throw new IllegalArgumentException("No schema specified")
}
} else {
// There are some existing files. Use them to infer the schema
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: "." at the end.

dataFrameBuilder(filesPresent.toArray).schema
}
}
}

/**
* Returns the maximum offset that can be retrieved from the source.
*
* `synchronized` on this method is for solving race conditions in tests. In the normal usage,
* there is no race here, so the cost of `synchronized` should be rare.
*/
private def fetchMaxOffset(): LongOffset = synchronized {
val filesPresent = fetchAllFiles()
val newFiles = new ArrayBuffer[String]()
filesPresent.foreach { file =>
if (!seenFiles.contains(file)) {
logDebug(s"new file: $file")
newFiles.append(file)
seenFiles.add(file)
} else {
logDebug(s"old file: $file")
}
}

if (newFiles.nonEmpty) {
maxBatchId += 1
writeBatch(maxBatchId, newFiles)
}

new LongOffset(maxBatchId)
}

/**
* For test only. Run `action` and return the current offset. When `action` is running, the method
* guarantee that the current offset won't be changed and no new batch will be emitted.
*/
def currentOffset(action: => Unit = {}): LongOffset = synchronized {
Copy link
Contributor

Choose a reason for hiding this comment

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

this is not a good name. currentOffset cannot take a body of code. Rather better to name this something like withBatchingLocked

fileStreamSource.withBatchingLocked {
    // do something
}

This can be separate method from simple currentOffset

action
new LongOffset(maxBatchId)
}

/**
* Returns the next batch of data that is available after `start`, if any is available.
*/
override def getNextBatch(start: Option[Offset]): Option[Batch] = {
val startId = start.map(_.asInstanceOf[LongOffset].offset).getOrElse(-1L)
val end = fetchMaxOffset()
val endId = end.offset

if (startId + 1 <= endId) {
val files = (startId + 1 to endId).filter(_ >= 0).flatMap { batchId =>
batchToMetadata.getOrElse(batchId, Nil)
}.toArray
logDebug(s"Return files from batches ${startId + 1}:$endId")
logDebug(s"Streaming ${files.mkString(", ")}")
Some(new Batch(end, dataFrameBuilder(files)))
}
else {
None
}
}

private def fetchAllBatchFiles(): Seq[FileStatus] = {
try fs.listStatus(new Path(metadataPath)) catch {
case _: java.io.FileNotFoundException =>
fs.mkdirs(new Path(metadataPath))
Seq.empty
}
}

private def fetchAllFiles(): Seq[String] = {
fs.listStatus(new Path(path))
.filterNot(_.getPath.getName.startsWith("_"))
.map(_.getPath.toUri.toString)
}

/**
* Write the metadata of a batch to disk. The file format is as follows:
*
* {{{
* SPARK_VERSION
Copy link
Contributor

Choose a reason for hiding this comment

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

I think we just want a metadata version here, since multiple spark versions will likely share the same format.

Copy link
Member Author

Choose a reason for hiding this comment

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

Added FileStreamSource.VERSION

Copy link
Contributor

Choose a reason for hiding this comment

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

Update this comment then.

* START
* -/a/b/c
* -/d/e/f
* ...
* END
* }}}
*
* Note: every file path starts with "-" so that we can know if a line is a file path easily.
*/
private def writeBatch(id: Int, files: Seq[String]): Unit = {
assert(files.nonEmpty, "create a new batch without any file")
val output = fs.create(new Path(metadataPath + "/" + id), true)
val writer = new PrintWriter(new OutputStreamWriter(output, UTF_8))
try {
// scalastyle:off println
writer.println(FileStreamSource.VERSION)
writer.println(FileStreamSource.START_TAG)
files.foreach(file => writer.println("-" + file))
Copy link
Contributor

Choose a reason for hiding this comment

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

would be good to make "-" a field like START_TAG. Say, PATH_PREFIX?

writer.println(FileStreamSource.END_TAG)
// scalastyle:on println
} finally {
writer.close()
}
batchToMetadata(id) = files
}

/** Read the file names of the specified batch id from the metadata file */
private def readBatch(id: Int): Seq[String] = {
val input = fs.open(new Path(metadataPath + "/" + id))
try {
FileStreamSource.readBatch(input)
} finally {
input.close()
}
}
}

object FileStreamSource {

private val START_TAG = "START"
private val END_TAG = "END"
val VERSION = "FILESTREAM1"
Copy link
Contributor

Choose a reason for hiding this comment

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

FILESTREAM_V1


/**
* Parse a metadata file and return the content. If the metadata file is corrupted, it will return
* 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

.drop(1) // The first line is version, just drop it
if (lines.isEmpty) {
return Nil
}
if (lines.head != "START") {
return Nil
}
if (lines.last != END_TAG) {
return Nil
}
lines.slice(1, lines.length - 1).map(_.drop(1)) // Drop character "-"
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection
import org.apache.spark.sql.execution.{FileRelation, RDDConversions}
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.streaming.{Sink, Source}
import org.apache.spark.sql.execution.streaming.{FileStreamSource, Sink, Source}
import org.apache.spark.sql.types.{StringType, StructType}
import org.apache.spark.util.SerializableConfiguration

Expand Down Expand Up @@ -130,8 +130,9 @@ trait SchemaRelationProvider {
trait StreamSourceProvider {
def createSource(
sqlContext: SQLContext,
parameters: Map[String, String],
schema: Option[StructType]): Source
schema: Option[StructType],
providerName: String,
parameters: Map[String, String]): Source
}

/**
Expand Down Expand Up @@ -168,7 +169,7 @@ trait StreamSinkProvider {
* @since 1.4.0
*/
@Experimental
trait HadoopFsRelationProvider {
trait HadoopFsRelationProvider extends StreamSourceProvider {
/**
* Returns a new base relation with the given parameters, a user defined schema, and a list of
* partition columns. Note: the parameters' keywords are case insensitive and this insensitivity
Expand All @@ -195,6 +196,28 @@ trait HadoopFsRelationProvider {
}
createRelation(sqlContext, paths, dataSchema, partitionColumns, parameters)
}

override def createSource(
sqlContext: SQLContext,
schema: Option[StructType],
providerName: String,
parameters: Map[String, String]): Source = {
val path = parameters("path")
val metadataPath = parameters.getOrElse("metadataPath", s"$path/_metadata")

def dataFrameBuilder(files: Array[String]): DataFrame = {
val relation = createRelation(
sqlContext,
files,
schema,
partitionColumns = None,
bucketSpec = None,
parameters)
DataFrame(sqlContext, LogicalRelation(relation))
}

new FileStreamSource(sqlContext, metadataPath, path, schema, providerName, dataFrameBuilder)
}
}

/**
Expand Down
2 changes: 2 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,8 @@ trait StreamTest extends QueryTest with Timeouts {

implicit class RichSource(s: Source) {
def toDF(): DataFrame = new DataFrame(sqlContext, StreamingRelation(s))

def toDS[A: Encoder](): Dataset[A] = new Dataset(sqlContext, StreamingRelation(s))
}

/** How long to wait for an active stream to catch up when checking a result. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,9 @@ object LastOptions {
class DefaultSource extends StreamSourceProvider with StreamSinkProvider {
override def createSource(
sqlContext: SQLContext,
parameters: Map[String, String],
schema: Option[StructType]): Source = {
schema: Option[StructType],
providerName: String,
parameters: Map[String, String]): Source = {
Copy link
Contributor

Choose a reason for hiding this comment

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

If we have to add stuff we should probably add a schema too.

LastOptions.parameters = parameters
LastOptions.schema = schema
new Source {
Expand Down
Loading