-
Notifications
You must be signed in to change notification settings - Fork 1
Initial draft of Streaming Dataframe infrastructure #21
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
Changes from 1 commit
e238911
d2706b5
7a3590f
c8a9238
dddd192
a0a1e7b
15bed31
89464a9
d133dbb
e3c4c83
90fa6d3
b1c1dc6
9205068
eab186d
cd575db
d233eae
3423dce
b0b20e5
f5d9642
f8911e4
6387781
6242bc2
95fd978
f928595
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,135 @@ | ||
| /* | ||
| * 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.{OutputStreamWriter, BufferedWriter} | ||
|
|
||
| import org.apache.hadoop.fs.{FileStatus, Path, FileSystem} | ||
| import org.apache.spark.{Logging, SparkContext} | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.SQLContext | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.types.{StringType, StructType} | ||
| import org.apache.spark.util.collection.OpenHashSet | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| class FileStream( | ||
| val sqlContext: SQLContext, | ||
| val metadataPath: String, | ||
| val path: String) extends Source with Logging { | ||
|
|
||
| import sqlContext.implicits._ | ||
|
|
||
| /** Returns the schema of the data from this source */ | ||
| override def schema: StructType = | ||
| StructType(Nil).add("value", StringType) | ||
|
|
||
| /** Returns the maximum offset that can be retrieved from the source. */ | ||
| override def offset: Offset = synchronized { | ||
| val filesPresent = fetchAllFiles() | ||
| val newFiles = new ArrayBuffer[String]() | ||
| filesPresent.foreach { file => | ||
| if (!seenFiles.contains(file)) { | ||
| logWarning(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 data between the `start` and `end` offsets. This function must always return | ||
| * the same set of data for any given pair of offsets. | ||
| */ | ||
| override def getSlice( | ||
| sqlContext: SQLContext, | ||
| start: Option[Offset], | ||
| end: Offset): RDD[InternalRow] = { | ||
| val startId = start.map(_.asInstanceOf[LongOffset].offset).getOrElse(0L) | ||
| val endId = end.asInstanceOf[LongOffset].offset | ||
|
|
||
| log.warn(s"Producing files from batches $start:$end") | ||
| if (endId == -1) { | ||
| sparkContext.emptyRDD | ||
| } else { | ||
| val batchFiles = ((startId + 1) to endId).map(i => s"$metadataPath$i") | ||
| log.warn(s"Batch files: $batchFiles") | ||
|
|
||
| val files = sqlContext | ||
| .read | ||
| .text(batchFiles: _*) | ||
| .as[String] | ||
| .collect() | ||
| log.warn(s"Streaming ${files.mkString(", ")}") | ||
| sqlContext.read.text(files: _*).queryExecution.executedPlan.execute() | ||
| } | ||
|
|
||
| } | ||
|
|
||
| def restart(): FileStream = { | ||
| new FileStream(sqlContext, metadataPath, path) | ||
| } | ||
|
|
||
| private def sparkContext = sqlContext.sparkContext | ||
|
|
||
| val fs = FileSystem.get(sparkContext.hadoopConfiguration) | ||
| val existingBatchFiles = fetchAllBatchFiles() | ||
| val existingBatchIds = existingBatchFiles.map(_.getPath.getName.toInt) | ||
| var maxBatchFile = if (existingBatchIds.isEmpty) -1 else existingBatchIds.max | ||
| val seenFiles = new OpenHashSet[String] | ||
|
|
||
| if (existingBatchFiles.nonEmpty) { | ||
| sqlContext.read | ||
| .text(existingBatchFiles.map(_.getPath.toString): _*) | ||
| .as[String] | ||
| .collect() | ||
| .foreach { file => | ||
| seenFiles.add(file) | ||
| } | ||
| } | ||
|
|
||
| private def fetchAllBatchFiles(): Seq[FileStatus] = { | ||
| fs.listStatus(new Path(metadataPath)) | ||
| } | ||
|
|
||
| private def fetchAllFiles(): Seq[String] = { | ||
| fs.listStatus(new Path(path)).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() | ||
| logWarning(s"Wrote batch file $path") | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ import scala.collection.mutable | |
|
|
||
| class StreamProgress extends Serializable { | ||
|
Collaborator
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. How will external sinks create this object if this neither takes a map of offsets in constructor, nor can the update() be called.
Owner
Author
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. I'll add a TODO for serialization API. |
||
| private val currentOffsets = new mutable.HashMap[Source, Offset] | ||
| with mutable.SynchronizedMap[Source, Offset] | ||
|
|
||
| def isEmpty: Boolean = currentOffsets.filterNot(_._2.isEmpty).isEmpty | ||
|
|
||
|
|
@@ -34,6 +35,7 @@ class StreamProgress extends Serializable { | |
|
|
||
| def apply(source: Source): Offset = currentOffsets(source) | ||
| def get(source: Source): Option[Offset] = currentOffsets.get(source) | ||
| def contains(source: Source): Boolean = currentOffsets.contains(source) | ||
|
|
||
| def ++(updates: Map[Source, Offset]): StreamProgress = { | ||
| val updated = new StreamProgress | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -37,26 +37,31 @@ import org.apache.spark.storage.{StreamBlockId, BlockId} | |
|
|
||
| object MemoryStream { | ||
| protected val currentBlockId = new AtomicInteger(0) | ||
| protected val memoryStreamId = new AtomicInteger(0) | ||
|
|
||
| def apply[A : Encoder]: MemoryStream[A] = | ||
| new MemoryStream[A](encoderFor[A].schema.toAttributes) | ||
| new MemoryStream[A](memoryStreamId.getAndIncrement()) | ||
| } | ||
|
|
||
| case class MemoryStream[A : Encoder](output: Seq[Attribute]) extends LeafNode with Source { | ||
| case class MemoryStream[A : Encoder](id: Int) extends Source with Logging { | ||
| protected val encoder = encoderFor[A] | ||
| protected val logicalPlan = StreamingRelation(this) | ||
| protected val output = logicalPlan.output | ||
| protected var blocks = new ArrayBuffer[BlockId] | ||
| protected var currentOffset: LongOffset = new LongOffset(-1) | ||
| protected val encoder = encoderFor[A] | ||
|
|
||
| protected def blockManager = SparkEnv.get.blockManager | ||
|
|
||
| def schema: StructType = encoder.schema | ||
|
|
||
| def offset: Offset = currentOffset | ||
|
|
||
| def toDS()(implicit sqlContext: SQLContext): Dataset[A] = { | ||
| new Dataset(sqlContext, this) | ||
| new Dataset(sqlContext, logicalPlan) | ||
| } | ||
|
|
||
| def toDF()(implicit sqlContext: SQLContext): DataFrame = { | ||
| new DataFrame(sqlContext, this) | ||
| new DataFrame(sqlContext, logicalPlan) | ||
| } | ||
|
|
||
| def addData(data: TraversableOnce[A]): Offset = { | ||
|
|
@@ -73,15 +78,17 @@ case class MemoryStream[A : Encoder](output: Seq[Attribute]) extends LeafNode wi | |
| } | ||
| } | ||
|
|
||
| def getSlice(sqlContext: SQLContext, start: Offset, end: Offset): RDD[InternalRow] = { | ||
| def getSlice(sqlContext: SQLContext, start: Option[Offset], end: Offset): RDD[InternalRow] = { | ||
| val newBlocks = | ||
| blocks.slice( | ||
| start.asInstanceOf[LongOffset].offset.toInt + 1, | ||
| start.map(_.asInstanceOf[LongOffset]).getOrElse(LongOffset(-1)).offset.toInt + 1, | ||
| end.asInstanceOf[LongOffset].offset.toInt + 1).toArray | ||
| logDebug(s"Running [$start, $end] on blocks ${newBlocks.mkString(", ")}") | ||
| new BlockRDD[InternalRow](sqlContext.sparkContext, newBlocks) | ||
| } | ||
|
|
||
| def restart(): Source = this | ||
|
Collaborator
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. What is this for if this just returns
Owner
Author
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. left over from the file tests where you need to restart in testing. removed |
||
|
|
||
| override def toString: String = s"MemoryStream[${output.mkString(",")}]" | ||
| } | ||
|
|
||
|
|
||
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.
nit: volatile and private[streaming]?