Skip to content

Commit

Permalink
[SC-5557] Initial implementation of directory level atomicity (omits …
Browse files Browse the repository at this point in the history
…delete hook)

## What changes were proposed in this pull request?

This implements a file commit protocol optimized for cloud storage. Files are written directly to their final locations. Their commit status is determined by the presence of specially named marker files.

Job commit proceeds as follows:

 1) When tasks request a file to write, we create a `_started_$txnId` marker in the output
    directory. The output files are hidden from readers while the start transaction marker is
    present.
 2) We commit the job by replacing the _started marker with a `_committed_$txnId` file that
    contains a list of files added and removed in that directory.

 The protocol is fail-open. That is, if a start marker is not present, we assume the file is committed.

 Note that this is only atomic per-directory, and may suffer from update anomalies if there
 are multiple concurrent writers.

 To clean up garbage files, we provide the sql VACUUM command that takes a time horizon.

 `VACUUM '/path/to/directory' [RETAIN <num> HOURS]` will remove garbage / uncommitted files after the specified number of hours, defaulting to 48.

### Config flags:
spark.sql.sources.commitProtocolClass -- commit protocol (default DatabricksAtomicCommitProtocol)
com.databricks.sql.enableFilterUncommitted -- whether to enable the read protocol (default true)
com.databricks.sql.ignoreCorruptCommitMarkers -- whether to ignore unreadable commit markers rather than raising error (default false)

## How was this patch tested?

Unit tests. TODO(ekl): randomized tests in a follow-up to verify correctness

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes apache#164 from ericl/directory-atomicity.
  • Loading branch information
ericl committed Jan 17, 2017
1 parent dae6c6f commit 227ff64
Show file tree
Hide file tree
Showing 15 changed files with 1,048 additions and 103 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -150,6 +150,7 @@ statement
| SET ROLE .*? #failNativeCommand
| SET .*? #setConfiguration
| RESET #resetConfiguration
| VACUUM path=STRING (RETAIN number HOURS)? #vacuumPath
| unsupportedHiveNativeCommands .*? #failNativeCommand
;

Expand Down Expand Up @@ -701,6 +702,7 @@ nonReserved
| AND | CASE | CAST | DISTINCT | DIV | ELSE | END | FUNCTION | INTERVAL | MACRO | OR | STRATIFY | THEN
| UNBOUNDED | WHEN
| DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT | CURRENT_DATE | CURRENT_TIMESTAMP
| VACUUM | RETAIN | HOURS
;

SELECT: 'SELECT';
Expand Down Expand Up @@ -810,6 +812,9 @@ START: 'START';
TRANSACTION: 'TRANSACTION';
COMMIT: 'COMMIT';
ROLLBACK: 'ROLLBACK';
VACUUM: 'VACUUM';
RETAIN: 'RETAIN';
HOURS: 'HOURS';
MACRO: 'MACRO';

IF: 'IF';
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,240 @@
/* Copyright © 2016 Databricks, Inc.
*
* Portions of this software incorporate or are derived from software contained within Apache Spark,
* and this modified software differs from the Apache Spark software provided under the Apache
* License, Version 2.0, a copy of which you may obtain at
* http://www.apache.org/licenses/LICENSE-2.0
*/

package org.apache.spark.sql.transaction

import java.io._
import java.nio.charset.StandardCharsets

import scala.collection.mutable
import scala.util.control.NonFatal

import org.apache.hadoop.fs._
import org.apache.hadoop.mapreduce._
import org.json4s.NoTypeHints
import org.json4s.jackson.Serialization

import org.apache.spark.SparkEnv
import org.apache.spark.internal.Logging
import org.apache.spark.internal.io.FileCommitProtocol
import org.apache.spark.sql.SparkSession

/**
* File commit protocol optimized for cloud storage. Files are written directly to their final
* locations. Their commit status is determined by the presence of specially named marker files.
*
* Job commit proceeds as follows:
*
* 1) When tasks request a file to write, we create a `_started_$txnId` marker in the output
* directory. The output files, which have $txnId embedded in their name, are hidden from
* readers while the start marker is present.
* 2) We commit the job by creating a new `_committed_$txnId` marker that contains a list of
* files added and removed in that directory.
*
* Note that this is only atomic per-directory, and that we only provide snapshot isolation and
* not serializability.
*/
class DatabricksAtomicCommitProtocol(jobId: String, path: String)
extends FileCommitProtocol with Serializable with Logging {

import FileCommitProtocol._
import DatabricksAtomicReadProtocol._
import DatabricksAtomicCommitProtocol._

// Globally unique alphanumeric string. We decouple this from jobId for possible future use.
private val txnId: TxnId = math.abs(scala.util.Random.nextLong).toString

// The list of files staged by this committer. These are collected to the driver on task commit.
private val stagedFiles = mutable.Set[String]()

// The list of files staged for deletion by the driver.
@transient private val stagedDeletions = mutable.Set[Path]()

override def newTaskTempFile(
taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = {
if (dir.isDefined) {
newTaskTempFileAbsPath(taskContext, new Path(path, dir.get).toString, ext)
} else {
newTaskTempFileAbsPath(taskContext, new Path(path).toString, ext)
}
}

override def newTaskTempFileAbsPath(
taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = {
val filename = getFilename(taskContext, ext)
val finalPath = new Path(absoluteDir, filename)
val fs = finalPath.getFileSystem(taskContext.getConfiguration)
val startMarker = new Path(finalPath.getParent, new Path(s"_started_$txnId"))
if (!fs.exists(startMarker)) {
fs.create(startMarker, true).close()
logDebug("Created start marker: " + startMarker)
}
stagedFiles += finalPath.toString
finalPath.toString
}

private def getFilename(taskContext: TaskAttemptContext, ext: String): String = {
// Note that %05d does not truncate the split number, so if we have more than 100000 tasks,
// the file name is fine and won't overflow.
val split = taskContext.getTaskAttemptID.getTaskID.getId

// Include the job and task attempt ids so that file writes never collide.
val taskAttemptId = taskContext.getTaskAttemptID.getId

// e.g. part-00001-tid-177723428-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb-0_00003.gz.parquet
f"part-$split%05d-tid-$txnId-$jobId-$taskAttemptId$ext"
}

override def setupJob(jobContext: JobContext): Unit = {
val root = new Path(path)
root.getFileSystem(jobContext.getConfiguration).mkdirs(root)
}

override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = {
logInfo("Committing job " + jobId)
val root = new Path(path)
val fs = root.getFileSystem(jobContext.getConfiguration)
def qualify(path: Path): Path = path.makeQualified(fs.getUri, fs.getWorkingDirectory)

// Collects start markers and staged task files.
taskCommits.foreach { t =>
val task = t.obj.asInstanceOf[this.type]
stagedFiles ++= task.stagedFiles
}

val addedByDir = stagedFiles.toSeq.map(new Path(_)).map(qualify)
.groupBy(_.getParent).map(kv => (kv._1, kv._2.map(_.getName.toString)))

val removedByDir = stagedDeletions.toSeq.map(qualify)
.groupBy(_.getParent).map(kv => (kv._1, kv._2.map(_.getName.toString)))

// Commit each updated directory in parallel.
val dirs = (addedByDir.keys ++ removedByDir.keys).toSet.par
dirs.tasksupport = DatabricksAtomicCommitProtocol.tasksupport
dirs.foreach { dir =>
val commitMarker = new Path(dir, s"_committed_$txnId")
val output = fs.create(commitMarker)
try {
serializeFileChanges(
addedByDir.getOrElse(dir, Nil), removedByDir.getOrElse(dir, Nil), output)
} finally {
output.close()
}
// We don't delete the start marker here since from a correctness perspective, it is
// possible a concurrent reader sees neither the start nor end marker even with a re-list
}
logInfo("Job commit completed for " + jobId)
}

override def abortJob(jobContext: JobContext): Unit = {
/* no-op */
}

override def setupTask(taskContext: TaskAttemptContext): Unit = {
/* no-op */
}

override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = {
new TaskCommitMessage(this)
}

override def abortTask(taskContext: TaskAttemptContext): Unit = {
// We must leave the start markers since other stray tasks may be writing to this same
// directory, and we need to ensure their files stay hidden.
stagedFiles.map(new Path(_)).foreach { f =>
val fs = f.getFileSystem(taskContext.getConfiguration)
fs.delete(f, false)
}
}
}

object DatabricksAtomicCommitProtocol extends Logging {
import DatabricksAtomicReadProtocol._

private val sparkSession = SparkSession.builder.getOrCreate()

import scala.collection.parallel.ThreadPoolTaskSupport
import java.util.concurrent.{LinkedBlockingQueue, ThreadFactory, ThreadPoolExecutor, TimeUnit}

private lazy val tasksupport = new ThreadPoolTaskSupport({
val pool = new ThreadPoolExecutor(
100,
100,
100L,
TimeUnit.SECONDS,
new LinkedBlockingQueue[Runnable])
pool.setThreadFactory(new ThreadFactory {
override def newThread(task: Runnable): Thread = {
val thread = new Thread(task, "DatabricksAtomicCommitProtocolWorker")
thread.setDaemon(true)
thread
}
})
pool
})

/**
* Traverses the given directories and cleans up uncommitted or garbage files and markers. A
* horizon may be specified beyond which we assume pending jobs have failed. Files written by
* those jobs will be removed as well. Vacuuming will be done in parallel if possible.
*
* @return the list of deleted files
*/
def vacuum(path: Path, horizon: Long): List[Path] = {
val fs = path.getFileSystem(sparkSession.sparkContext.hadoopConfiguration)
val (dirs, initialFiles) = fs.listStatus(path).partition(_.isDirectory)

def checkPositive(time: Long): Long = { assert(time > 0); time }
var deletedPaths: List[Path] = Nil
def delete(p: Path): Unit = {
deletedPaths ::= p
fs.delete(p, false)
}

val (state, resolvedFiles) = resolveCommitState(fs, path, initialFiles)

// remove uncommitted and timed-out file outputs
for (file <- resolvedFiles) {
file.getPath.getName match {
// we wait for a horizon to avoid killing Spark jobs using those files
case name if state.getDeletionTime(name) > 0 && state.getDeletionTime(name) < horizon =>
logInfo(s"Garbage collecting ${file.getPath} since it is marked as deleted.")
delete(file.getPath)

case name @ FILE_WITH_TXN_ID(txnId) if state.isCommitted(txnId) &&
!state.isFileCommitted(txnId, name) =>
logInfo(s"Garbage collecting ${file.getPath} since it was written by a failed task.")
delete(file.getPath)

case name @ FILE_WITH_TXN_ID(txnId) if !state.isCommitted(txnId) &&
checkPositive(state.getStartTime(txnId)) < horizon =>
logInfo(s"Garbage collecting ${file.getPath} since its job has timed out " +
s"(${state.getStartTime(txnId)} < $horizon).")
delete(file.getPath)

case STARTED_MARKER(txnId) if state.isCommitted(txnId) &&
checkPositive(file.getModificationTime) < horizon =>
logInfo(s"Garbage collecting start marker ${file.getPath} of committed job.")
delete(file.getPath)

case _ =>
}
}

// recurse
for (d <- dirs) {
deletedPaths :::= vacuum(d.getPath, horizon)
if (fs.listStatus(d.getPath).isEmpty) {
logInfo(s"Garbage collecting empty directory ${d.getPath}")
delete(d.getPath)
}
}

deletedPaths
}
}
Loading

0 comments on commit 227ff64

Please sign in to comment.