Skip to content

Commit

Permalink
cleanup
Browse files Browse the repository at this point in the history
  • Loading branch information
marmbrus committed Mar 22, 2016
1 parent e861a68 commit 8e8e4c6
Show file tree
Hide file tree
Showing 11 changed files with 240 additions and 207 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -94,8 +94,9 @@ trait ContinuousQuery {
/**
* Blocks until all available data in the source has been processed an committed to the sink.
* This method is intended for testing. Note that in the case of continually arriving data, this
* method may block forever. Additionally, it only works with sources that synchronously append
* data (i.e. `getOffset` will show the new data immediately upon addition).
* method may block forever. Additionally, this method is only guranteed to block until data that
* has been synchronously appended data to a [[org.apache.spark.sql.execution.streaming.Source]]
* prior to invocation. (i.e. `getOffset` must immediately reflect the addition).
*/
def processAllAvailable(): Unit

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,7 @@ import org.apache.spark.sql.execution.streaming.{Offset, StreamExecution}
*/
@Experimental
class ContinuousQueryException private[sql](
@transient
val query: ContinuousQuery,
@transient val query: ContinuousQuery,
val message: String,
val cause: Throwable,
val startOffset: Option[Offset] = None,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@ import java.util.ServiceLoader

import scala.collection.JavaConverters._
import scala.language.{existentials, implicitConversions}
import scala.util.control.NonFatal
import scala.util.{Failure, Success, Try}
import scala.util.control.NonFatal

import org.apache.hadoop.fs.Path

Expand Down Expand Up @@ -192,6 +192,10 @@ case class DataSource(
}
}

/**
* Returns true if there is a single path that has a metadata log indicating which files should
* be read.
*/
def hasMetadata(path: Seq[String]): Boolean = {
path match {
case Seq(singlePath) =>
Expand All @@ -200,7 +204,6 @@ case class DataSource(
val fs = hdfsPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
val metadataPath = new Path(hdfsPath, FileStreamSink.metadataDir)
val res = fs.exists(metadataPath)
println(s"checking for metadata at $metadataPath : $res")
res
} catch {
case NonFatal(e) =>
Expand All @@ -225,6 +228,8 @@ case class DataSource(
case (_: RelationProvider, Some(_)) =>
throw new AnalysisException(s"$className does not allow user-specified schemas.")

// We are reading from the results of a streaming query. Load files from the metadata log
// instead of listing them using HDFS APIs.
case (format: FileFormat, _)
if hasMetadata(caseInsensitiveOptions.get("path").toSeq ++ paths) =>
val basePath = new Path((caseInsensitiveOptions.get("path").toSeq ++ paths).head)
Expand All @@ -250,6 +255,7 @@ case class DataSource(
format,
options)

// This is a non-streaming file based datasource.
case (format: FileFormat, _) =>
val allPaths = caseInsensitiveOptions.get("path") ++ paths
val globbedPaths = allPaths.flatMap { path =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ private[parquet] class CatalystWriteSupport extends WriteSupport[InternalRow] wi
private type ValueWriter = (SpecializedGetters, Int) => Unit

// Schema of the `InternalRow`s to be written
var schema: StructType = _
private var schema: StructType = _

// `ValueWriter`s for all fields of the schema
private var rootFieldWriters: Seq[ValueWriter] = _
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,57 +19,23 @@ package org.apache.spark.sql.execution.streaming

import java.util.UUID

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.datasources.PartitionSpec
import org.apache.spark.sql.sources.{Partition, FileCatalog, FileFormat}

import scala.collection.mutable.ArrayBuffer

import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
import org.apache.hadoop.fs.Path

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

object FileStreamSink {
// The name of the subdirectory that is used to store metadata about which files are valid.
val metadataDir = "_spark_metadata"
}

class StreamFileCatalog(sqlContext: SQLContext, path: Path) extends FileCatalog with Logging {
val metadataDirectory = new Path(path, FileStreamSink.metadataDir)
logInfo(s"Reading log from $metadataDirectory")
val metadataLog = new HDFSMetadataLog[Seq[String]](sqlContext, metadataDirectory.toUri.toString)
val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)

override def paths: Seq[Path] = path :: Nil

override def partitionSpec(): PartitionSpec = PartitionSpec(StructType(Nil), Nil)

/**
* Returns all valid files grouped into partitions when the data is partitioned. If the data is
* unpartitioned, this will return a single partition with not partition values.
*
* @param filters the filters used to prune which partitions are returned. These filters must
* only refer to partition columns and this method will only return files
* where these predicates are guaranteed to evaluate to `true`. Thus, these
* filters will not need to be evaluated again on the returned data.
*/
override def listFiles(filters: Seq[Expression]): Seq[Partition] =
Partition(InternalRow.empty, allFiles()) :: Nil

override def getStatus(path: Path): Array[FileStatus] = fs.listStatus(path)

override def refresh(): Unit = {}

override def allFiles(): Seq[FileStatus] = {
fs.listStatus(metadataLog.get(None, None).flatMap(_._2).map(new Path(_)))
}
}

/**
*
* A sink that writes out results to parquet files. Each batch is written out to a unique
* directory. After all of the files in a batch have been succesfully written, the list of
* file paths is appended to the log atomically. In the case of partial failures, some duplicate
* data may be present in the target directory, but only one copy of each file will be present
* in the log.
*/
class FileStreamSink(
sqlContext: SQLContext,
Expand All @@ -78,49 +44,36 @@ class FileStreamSink(

val basePath = new Path(path)
val logPath = new Path(basePath, FileStreamSink.metadataDir)
logInfo(s"Logging to $logPath")
val fileLog = new HDFSMetadataLog[Seq[String]](sqlContext, logPath.toUri.toString)

override def addBatch(batchId: Long, data: DataFrame): Unit = {
logInfo(s"STARTING BATCH COMMIT $batchId")
if (fileLog.get(batchId).isDefined) {
logInfo(s"Skipping already commited batch $batchId")
logInfo(s"Skipping already committed batch $batchId")
} else {
val files = writeFiles(data)

println(s"Wrote ${files.size}")

if (fileLog.add(batchId, files)) {
logInfo(s"Wrote batch $batchId")
logInfo(s"Committed batch $batchId")
} else {
logInfo(s"Someone beat us to batch $batchId")
logWarning(s"Race while writing batch $batchId")
}
}
logInfo(s"DONE BATCH COMMIT $batchId")
}

/** Writes the [[DataFrame]] to a UUID-named dir, returning the list of files paths. */
private def writeFiles(data: DataFrame): Seq[String] = {
val ctx = sqlContext
val outputDir = path
val format = fileFormat
val schema = data.schema

/*
data.queryExecution.executedPlan.execute().mapPartitions { rows =>
val basePath = new Path(outputDir)
val file = new Path(basePath, UUID.randomUUID().toString)
val writer = format.openWriter(ctx, file.toUri.toString, schema)
rows.foreach(writer.write)
Iterator(file.toString)
}.collect().toSeq
*/

val file = new Path(basePath, UUID.randomUUID().toString).toUri.toString
println(s"writing to uuid at $file")
data.write.parquet(file)
sqlContext.read
.schema(data.schema)
.parquet(file).inputFiles
.parquet(file)
.inputFiles
.map(new Path(_))
.filterNot(_.getName.startsWith("_"))
.map(_.toUri.toString)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ class FileStreamSource(

assert(startId <= endId)
val files = metadataLog.get(Some(startId + 1), Some(endId)).map(_._2).flatten
logError(s"Processing ${files.length} files from ${startId + 1}:$endId")
logInfo(s"Processing ${files.length} files from ${startId + 1}:$endId")
logDebug(s"Streaming ${files.mkString(", ")}")
dataFrameBuilder(files)

Expand All @@ -127,7 +127,7 @@ class FileStreamSource(
.filterNot(_.getPath.getName.startsWith("_"))
.map(_.getPath.toUri.toString)
val endTime = System.nanoTime()
logError(s"Listed ${files.size} in ${(endTime.toDouble - startTime) / 1000000}ms")
logDebug(s"Listed ${files.size} in ${(endTime.toDouble - startTime) / 1000000}ms")
files
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ class StreamExecution(
s"Query $name terminated with exception: ${e.getMessage}",
e,
Some(committedOffsets.toCompositeOffset(sources)))
logDebug(s"Query $name terminated with error", e)
logError(s"Query $name terminated with error", e)
} finally {
state = TERMINATED
sqlContext.streams.notifyQueryTermination(StreamExecution.this)
Expand Down Expand Up @@ -196,7 +196,7 @@ class StreamExecution(
}

case None => // We are starting this stream for the first time.
logDebug(s"Starting new continuous query.")
logInfo(s"Starting new continuous query.")
currentBatchId = 0
commitAndConstructNextBatch()
}
Expand Down Expand Up @@ -303,7 +303,7 @@ class StreamExecution(
}

val batchTime = (System.nanoTime() - startTime).toDouble / 1000000
logDebug(s"Completed up to $availableOffsets in ${batchTime}ms")
logInfo(s"Completed up to $availableOffsets in ${batchTime}ms")
postEvent(new QueryProgress(this))
}

Expand All @@ -323,7 +323,7 @@ class StreamExecution(
microBatchThread.interrupt()
microBatchThread.join()
}
logDebug(s"Query $name was stopped")
logInfo(s"Query $name was stopped")
}

/**
Expand All @@ -334,12 +334,24 @@ class StreamExecution(
def notDone = !committedOffsets.contains(source) || committedOffsets(source) < newOffset

while (notDone) {
logDebug(s"Waiting until $newOffset at $source")
logInfo(s"Waiting until $newOffset at $source")
awaitBatchLock.synchronized { awaitBatchLock.wait(100) }
}
logDebug(s"Unblocked at $newOffset for $source")
}

/** A flag to indicate that a batch has completed with no new data available. */
@volatile private var noNewData = false

override def processAllAvailable(): Unit = {
noNewData = false
while (!noNewData) {
awaitBatchLock.synchronized { awaitBatchLock.wait(10000) }
if (streamDeathCause != null) { throw streamDeathCause }
}
if (streamDeathCause != null) { throw streamDeathCause }
}

override def awaitTermination(): Unit = {
if (state == INITIALIZED) {
throw new IllegalStateException("Cannot wait for termination on a query that has not started")
Expand Down Expand Up @@ -390,16 +402,6 @@ class StreamExecution(
case object INITIALIZED extends State
case object ACTIVE extends State
case object TERMINATED extends State

var noNewData = false
override def processAllAvailable(): Unit = {
noNewData = false
while (!noNewData) {
awaitBatchLock.synchronized { awaitBatchLock.wait(10000) }
if (streamDeathCause != null) { throw streamDeathCause }
}
if (streamDeathCause != null) { throw streamDeathCause }
}
}

private[sql] object StreamExecution {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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 org.apache.hadoop.fs.{FileStatus, Path}

import org.apache.spark.internal.Logging
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.datasources.PartitionSpec
import org.apache.spark.sql.sources.{FileCatalog, Partition}
import org.apache.spark.sql.types.StructType

class StreamFileCatalog(sqlContext: SQLContext, path: Path) extends FileCatalog with Logging {
val metadataDirectory = new Path(path, FileStreamSink.metadataDir)
logInfo(s"Reading streaming file log from $metadataDirectory")
val metadataLog = new HDFSMetadataLog[Seq[String]](sqlContext, metadataDirectory.toUri.toString)
val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)

override def paths: Seq[Path] = path :: Nil

override def partitionSpec(): PartitionSpec = PartitionSpec(StructType(Nil), Nil)

/**
* Returns all valid files grouped into partitions when the data is partitioned. If the data is
* unpartitioned, this will return a single partition with not partition values.
*
* @param filters the filters used to prune which partitions are returned. These filters must
* only refer to partition columns and this method will only return files
* where these predicates are guaranteed to evaluate to `true`. Thus, these
* filters will not need to be evaluated again on the returned data.
*/
override def listFiles(filters: Seq[Expression]): Seq[Partition] =
Partition(InternalRow.empty, allFiles()) :: Nil

override def getStatus(path: Path): Array[FileStatus] = fs.listStatus(path)

override def refresh(): Unit = {}

override def allFiles(): Seq[FileStatus] = {
fs.listStatus(metadataLog.get(None, None).flatMap(_._2).map(new Path(_)))
}
}
Loading

0 comments on commit 8e8e4c6

Please sign in to comment.