-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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
Changes from 1 commit
a2784ff
6a90c55
9f5967f
937b86f
2af6fc8
f38153b
1ccea0f
ca1e6b8
d911963
ce0556d
9fd21c2
9a1042c
91a2b74
1ffee5f
fb0e3f9
07e2ddd
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 |
---|---|---|
@@ -0,0 +1,63 @@ | ||
/* | ||
* 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.nio.ByteBuffer | ||
|
||
import org.apache.hadoop.fs.{FileSystem, Path} | ||
|
||
import org.apache.spark.Logging | ||
import org.apache.spark.serializer.JavaSerializer | ||
import org.apache.spark.sql.SQLContext | ||
|
||
/** | ||
* A very simple sink that stores received data on the filesystem as a text file. | ||
* This is not atomic. | ||
*/ | ||
class FileStreamSink( | ||
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. If possible it would be nice to do this in a different PR. |
||
sqlContext: SQLContext, | ||
metadataPath: String, | ||
path: String) extends Sink with Logging { | ||
|
||
private def sparkContext = sqlContext.sparkContext | ||
private val fs = FileSystem.get(sparkContext.hadoopConfiguration) | ||
private val serializer = new JavaSerializer(sqlContext.sparkContext.conf).newInstance() | ||
|
||
override def currentOffset: Option[Offset] = { | ||
try { | ||
val buffer = new Array[Byte](10240) | ||
val stream = fs.open(new Path(metadataPath)) | ||
val size = stream.read(buffer) | ||
val shrunk = ByteBuffer.wrap(buffer.take(size)) | ||
Some(serializer.deserialize[Offset](shrunk)) | ||
} catch { | ||
case _: java.io.FileNotFoundException => | ||
None | ||
} | ||
} | ||
|
||
// TODO: this is not atomic. | ||
override def addBatch(batch: Batch): Unit = { | ||
batch.data.write.mode("append").text(path) | ||
val offset = serializer.serialize(batch.end) | ||
val stream = fs.create(new Path(metadataPath), true) | ||
stream.write(offset.array()) | ||
stream.close() | ||
logInfo(s"Committed batch ${batch.end}") | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,141 @@ | ||
/* | ||
* 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.{BufferedWriter, OutputStreamWriter} | ||
|
||
import scala.collection.mutable.ArrayBuffer | ||
|
||
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. | ||
*/ | ||
class FileStreamSource( | ||
sqlContext: SQLContext, | ||
metadataPath: String, | ||
path: String, | ||
dataSchema: Option[StructType], | ||
dataFrameBuilder: Array[String] => DataFrame) extends Source with Logging { | ||
|
||
import sqlContext.implicits._ | ||
|
||
/** Returns the schema of the data from this source */ | ||
override def schema: StructType = dataSchema.getOrElse(new StructType().add("value", StringType)) | ||
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 We should also add tests that would catch a problem here. 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. 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. |
||
|
||
/** Returns the maximum offset that can be retrieved from the source. */ | ||
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) { | ||
maxBatchFile += 1 | ||
writeBatch(maxBatchFile, newFiles) | ||
} | ||
|
||
new LongOffset(maxBatchFile) | ||
} | ||
|
||
/** | ||
* 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(0L) | ||
val end = fetchMaxOffset() | ||
val endId = end.offset | ||
|
||
val batchFiles = (startId to endId).filter(_ >= 0).map(i => s"$metadataPath/$i") | ||
if (!(batchFiles.isEmpty || start == Some(end))) { | ||
logDebug(s"Producing files from batches $start:$endId") | ||
logDebug(s"Batch files: $batchFiles") | ||
|
||
// Probably does not need to be a spark job... | ||
val files = sqlContext | ||
.read | ||
.text(batchFiles: _*) | ||
.as[String] | ||
.collect() | ||
logDebug(s"Streaming ${files.mkString(", ")}") | ||
Some(new Batch(end, dataFrameBuilder(files))) | ||
} else { | ||
None | ||
} | ||
} | ||
|
||
def restart(): FileStreamSource = { | ||
new FileStreamSource(sqlContext, metadataPath, path, dataSchema, dataFrameBuilder) | ||
} | ||
|
||
private def sparkContext = sqlContext.sparkContext | ||
|
||
private val fs = FileSystem.get(sparkContext.hadoopConfiguration) | ||
private val existingBatchFiles = fetchAllBatchFiles() | ||
private val existingBatchIds = existingBatchFiles.map(_.getPath.getName.toInt) | ||
private var maxBatchFile = if (existingBatchIds.isEmpty) -1 else existingBatchIds.max | ||
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. nit: Isnt this |
||
private val seenFiles = new OpenHashSet[String] | ||
|
||
if (existingBatchFiles.nonEmpty) { | ||
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 you add some comments on what this Spark job is for? |
||
sqlContext.read | ||
.text(existingBatchFiles.map(_.getPath.toString): _*) | ||
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. shouldnt these be two indented rather than four indented? |
||
.as[String] | ||
.collect() | ||
.foreach { file => | ||
seenFiles.add(file) | ||
} | ||
} | ||
|
||
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) | ||
} | ||
|
||
private def writeBatch(id: Int, files: Seq[String]): Unit = { | ||
val path = new Path(metadataPath + "/" + id) | ||
val fs = FileSystem.get(sparkContext.hadoopConfiguration) | ||
val writer = new BufferedWriter(new OutputStreamWriter(fs.create(path, true))) | ||
files.foreach { file => | ||
writer.write(file) | ||
writer.write("\n") | ||
} | ||
writer.close() | ||
logDebug(s"Wrote batch file $path") | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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.{FileStreamSink, FileStreamSource, Sink, Source} | ||
import org.apache.spark.sql.types.{StringType, StructType} | ||
import org.apache.spark.util.SerializableConfiguration | ||
|
||
|
@@ -130,8 +130,9 @@ trait SchemaRelationProvider { | |
trait StreamSourceProvider { | ||
def createSource( | ||
sqlContext: SQLContext, | ||
parameters: Map[String, String], | ||
schema: Option[StructType]): Source | ||
partitionColumns: Option[StructType], | ||
dataSchema: Option[StructType], | ||
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. nit: could we order these the same as below: dataSchema, partitionColumns, parameters |
||
parameters: Map[String, String]): Source | ||
} | ||
|
||
/** | ||
|
@@ -168,7 +169,7 @@ trait StreamSinkProvider { | |
* @since 1.4.0 | ||
*/ | ||
@Experimental | ||
trait HadoopFsRelationProvider { | ||
trait HadoopFsRelationProvider extends StreamSourceProvider with StreamSinkProvider { | ||
/** | ||
* 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 | ||
|
@@ -195,6 +196,38 @@ trait HadoopFsRelationProvider { | |
} | ||
createRelation(sqlContext, paths, dataSchema, partitionColumns, parameters) | ||
} | ||
|
||
override def createSource( | ||
sqlContext: SQLContext, | ||
partitionColumns: Option[StructType], | ||
dataSchema: Option[StructType], | ||
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, | ||
dataSchema, | ||
partitionColumns, | ||
bucketSpec = None, | ||
parameters) | ||
DataFrame(sqlContext, LogicalRelation(relation)) | ||
} | ||
|
||
new FileStreamSource(sqlContext, metadataPath, path, dataSchema, dataFrameBuilder) | ||
} | ||
|
||
override def createSink( | ||
sqlContext: SQLContext, | ||
parameters: Map[String, String], | ||
partitionColumns: Seq[String]): Sink = { | ||
val path = parameters("path") | ||
val metadataPath = parameters.getOrElse("metadataPath", s"$path/_metadata") | ||
|
||
new FileStreamSink(sqlContext, metadataPath, path) | ||
} | ||
} | ||
|
||
/** | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -33,10 +33,11 @@ object LastOptions { | |
class DefaultSource extends StreamSourceProvider with StreamSinkProvider { | ||
override def createSource( | ||
sqlContext: SQLContext, | ||
parameters: Map[String, String], | ||
schema: Option[StructType]): Source = { | ||
partitionColumns: Option[StructType], | ||
dataSchema: Option[StructType], | ||
parameters: Map[String, String]): Source = { | ||
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. If we have to add stuff we should probably add a schema too. |
||
LastOptions.parameters = parameters | ||
LastOptions.schema = schema | ||
LastOptions.schema = dataSchema | ||
new Source { | ||
override def getNextBatch(start: Option[Offset]): Option[Batch] = None | ||
override def schema: StructType = StructType(StructField("a", IntegerType) :: Nil) | ||
|
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.
Wait second... I think we only ever give the partition columns to a DataSource when it is coming from the hive metastore. Given that, I don't know if we actually need to do this at all for streaming. Could we simplify this?