-
Notifications
You must be signed in to change notification settings - Fork 28.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-14078] Streaming Parquet Based FileSink
This PR adds a new `Sink` implementation that writes out Parquet files. In order to correctly handle partial failures while maintaining exactly once semantics, the files for each batch are written out to a unique directory and then atomically appended to a metadata log. When a parquet based `DataSource` is initialized for reading, we first check for this log directory and use it instead of file listing when present. Unit tests are added, as well as a stress test that checks the answer after non-deterministic injected failures. Author: Michael Armbrust <michael@databricks.com> Closes #11897 from marmbrus/fileSink.
- Loading branch information
Showing
14 changed files
with
430 additions
and
15 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
81 changes: 81 additions & 0 deletions
81
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,81 @@ | ||
/* | ||
* 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.util.UUID | ||
|
||
import org.apache.hadoop.fs.Path | ||
|
||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.sql.{DataFrame, SQLContext} | ||
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" | ||
} | ||
|
||
/** | ||
* 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, | ||
path: String, | ||
fileFormat: FileFormat) extends Sink with Logging { | ||
|
||
private val basePath = new Path(path) | ||
private val logPath = new Path(basePath, FileStreamSink.metadataDir) | ||
private val fileLog = new HDFSMetadataLog[Seq[String]](sqlContext, logPath.toUri.toString) | ||
|
||
override def addBatch(batchId: Long, data: DataFrame): Unit = { | ||
if (fileLog.get(batchId).isDefined) { | ||
logInfo(s"Skipping already committed batch $batchId") | ||
} else { | ||
val files = writeFiles(data) | ||
if (fileLog.add(batchId, files)) { | ||
logInfo(s"Committed batch $batchId") | ||
} else { | ||
logWarning(s"Race while writing batch $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 | ||
|
||
val file = new Path(basePath, UUID.randomUUID().toString).toUri.toString | ||
data.write.parquet(file) | ||
sqlContext.read | ||
.schema(data.schema) | ||
.parquet(file) | ||
.inputFiles | ||
.map(new Path(_)) | ||
.filterNot(_.getName.startsWith("_")) | ||
.map(_.toUri.toString) | ||
} | ||
|
||
override def toString: String = s"FileSink[$path]" | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
59 changes: 59 additions & 0 deletions
59
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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(_))) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.