Skip to content

Commit

Permalink
[SC-5835][DIRECTORYCOMMIT] Move DirectoryCommitProtocol to its own pa…
Browse files Browse the repository at this point in the history
…ckage

## What changes were proposed in this pull request?
As part of the effort of clearly separating out Edge functionality, we're hereby moving all code related to the Directory Commit protocol to its own package.

A few things about this change are not ideal - had to:
  - import com.databricks in `PartitioningAwareFileIndex`
  - open up `DirectoryAtomicReadProtocol.testingFs` (for testig hack)
  - write ugly code for getting configs from `SparkEnv.conf`, because of not having access to `ConfigEntry`
  - duplicate a bunch of utility classes:  `Clock`, `ManualClock`, `SystemClock`, `ThreadUtils`

... but most of these (except the last) should hopefully be resolved by [SC-5838](https://databricks.atlassian.net/browse/SC-5838).

## How was this patch tested?
spark-sql tests

Author: Adrian Ionescu <adrian@databricks.com>

Closes apache#247 from adrian-ionescu/SC-5835.
  • Loading branch information
adrian-ionescu committed Mar 8, 2017
1 parent 0a21de2 commit 0985111
Show file tree
Hide file tree
Showing 15 changed files with 405 additions and 83 deletions.
2 changes: 1 addition & 1 deletion project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -853,7 +853,7 @@ object TestSettings {
javaOptions in Test += "-Dspark.ui.showConsoleProgress=false",
javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true",
javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=false",
javaOptions in Test += "-Dspark.sql.sources.commitProtocolClass=com.databricks.DatabricksAtomicCommitProtocol",
javaOptions in Test += "-Dspark.sql.sources.commitProtocolClass=com.databricks.sql.transaction.directory.DirectoryAtomicCommitProtocol",
javaOptions in Test += "-Dderby.system.durability=test",
javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark"))
.map { case (k,v) => s"-D$k=$v" }.toSeq,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@
package com.databricks.sql.acl

import com.databricks.sql.acl.Action._
import com.databricks.sql.transaction.VacuumTableCommand
import com.databricks.sql.transaction.directory.VacuumTableCommand

import org.apache.spark.sql.catalog.{Catalog => PublicCatalog}
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ package com.databricks.sql.parser
import scala.collection.JavaConverters._

import com.databricks.sql.parser.DatabricksSqlBaseParser._
import com.databricks.sql.transaction.VacuumTableCommand
import com.databricks.sql.transaction.directory.VacuumTableCommand

import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils}
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -6,25 +6,22 @@
* http://www.apache.org/licenses/LICENSE-2.0
*/

package org.apache.spark.sql.transaction
package com.databricks.sql.transaction.directory

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

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

import com.databricks.sql.DatabricksSQLConf._
import com.databricks.util.ThreadUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem => HadoopFileSystem, _}
import org.apache.hadoop.mapreduce._
import org.json4s.NoTypeHints
import org.json4s.jackson.Serialization

import org.apache.spark.internal.Logging
import org.apache.spark.internal.io.FileCommitProtocol
import org.apache.spark.sql.SparkSession
import org.apache.spark.util.ThreadUtils

/**
* File commit protocol optimized for cloud storage. Files are written directly to their final
Expand All @@ -41,12 +38,13 @@ import org.apache.spark.util.ThreadUtils
* Note that this is only atomic per-directory, and that we only provide snapshot isolation and
* not serializability.
*/
class DatabricksAtomicCommitProtocol(jobId: String, path: String)
class DirectoryAtomicCommitProtocol(jobId: String, path: String)
extends FileCommitProtocol with Serializable with Logging {

import DirectoryAtomicCommitProtocol._
import DirectoryAtomicReadProtocol._

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

// Globally unique alphanumeric string. We decouple this from jobId for possible future use.
private val txnId: TxnId = newTxnId()
Expand Down Expand Up @@ -202,8 +200,8 @@ class DatabricksAtomicCommitProtocol(jobId: String, path: String)
}
}

object DatabricksAtomicCommitProtocol extends Logging {
import DatabricksAtomicReadProtocol._
object DirectoryAtomicCommitProtocol extends Logging {
import DirectoryAtomicReadProtocol._

import scala.collection.parallel.ThreadPoolTaskSupport

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,28 +6,28 @@
* http://www.apache.org/licenses/LICENSE-2.0
*/

package org.apache.spark.sql.transaction
package com.databricks.sql.transaction.directory

import java.io.{File, FileNotFoundException, InputStream, InputStreamReader, IOException, OutputStream}
import java.io.{FileNotFoundException, InputStream, InputStreamReader, IOException, OutputStream}
import java.nio.charset.StandardCharsets

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

import com.databricks.sql.DatabricksSQLConf._
import com.databricks.util.{Clock, SystemClock, ThreadUtils}
import org.apache.hadoop.fs._
import org.json4s.NoTypeHints
import org.json4s.jackson.Serialization

import org.apache.spark.SparkEnv
import org.apache.spark.internal.Logging
import org.apache.spark.util.{Clock, SystemClock, ThreadUtils}

/**
* Read-side support for DatabricksAtomicCommitProtocol.
* Read-side support for DirectoryAtomicCommitProtocol.
*/
object DatabricksAtomicReadProtocol extends Logging {
object DirectoryAtomicReadProtocol extends Logging {
type TxnId = String

import scala.collection.parallel.ThreadPoolTaskSupport
Expand All @@ -41,11 +41,12 @@ object DatabricksAtomicReadProtocol extends Logging {

private implicit val formats = Serialization.formats(NoTypeHints)

// Visible for testing.
private[spark] var testingFs: Option[FileSystem] = None
// Visible because it's used as a hack in PartitioningAwareFileIndex.
var testingFs: Option[FileSystem] = None

// Visible for testing.
private[spark] var clock: Clock = new SystemClock
private[directory] var clock: Clock = new SystemClock


/**
* Given a directory listing, filters out files that are uncommitted. A file is considered
Expand All @@ -57,7 +58,9 @@ object DatabricksAtomicReadProtocol extends Logging {
def filterDirectoryListing(
fs: FileSystem, dir: Path, initialFiles: Seq[FileStatus]): Seq[FileStatus] = {
// we use SparkEnv for this escape-hatch flag since this may be called on executors
if (!SparkEnv.get.conf.get(DIRECTORY_COMMIT_FILTER_UNCOMMITTED)) {
if (!SparkEnv.get.conf.get(
DIRECTORY_COMMIT_FILTER_UNCOMMITTED.key,
DIRECTORY_COMMIT_FILTER_UNCOMMITTED.defaultValueString).toBoolean) {
return initialFiles
}

Expand Down Expand Up @@ -173,14 +176,16 @@ object DatabricksAtomicReadProtocol extends Logging {
* The same issue can occur with data file writes re-ordered after commit marker creation. In
* this situation we also must re-list if data files are suspected to be missing.
*/
private[transaction] def resolveCommitState(
def resolveCommitState(
fs: FileSystem,
dir: Path,
initialFiles: Seq[FileStatus]): (CommitState, Seq[FileStatus]) = {
val state = resolveCommitState0(fs, dir, initialFiles)

// Optimization: can assume the list request was atomic if the files have not changed recently.
val horizonMillis = SparkEnv.get.conf.get(DIRECTORY_COMMIT_WRITE_REORDERING_HORIZON_MS)
val horizonMillis = SparkEnv.get.conf.get(
DIRECTORY_COMMIT_WRITE_REORDERING_HORIZON_MS.key,
DIRECTORY_COMMIT_WRITE_REORDERING_HORIZON_MS.defaultValueString).toLong

if ((state.missingMarkers.nonEmpty || state.missingDataFiles.nonEmpty) &&
state.lastModified > clock.getTimeMillis - horizonMillis) {
Expand Down Expand Up @@ -287,7 +292,9 @@ object DatabricksAtomicReadProtocol extends Logging {

case NonFatal(e) =>
// we use SparkEnv for this escape-hatch flag since this may be called on executors
if (SparkEnv.get.conf.get(DIRECTORY_COMMIT_IGNORE_CORRUPT_MARKERS)) {
if (SparkEnv.get.conf.get(
DIRECTORY_COMMIT_IGNORE_CORRUPT_MARKERS.key,
DIRECTORY_COMMIT_IGNORE_CORRUPT_MARKERS.defaultValueString).toBoolean) {
logWarning("Failed to read job commit marker: " + stat, e)
corruptCommitMarkers.add(txnId)
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package com.databricks.sql.transaction
package com.databricks.sql.transaction.directory

import java.net.URI

Expand All @@ -25,7 +25,6 @@ import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.execution.command.RunnableCommand
import org.apache.spark.sql.transaction.DatabricksAtomicCommitProtocol
import org.apache.spark.sql.types._

case class VacuumTableCommand(
Expand All @@ -42,7 +41,7 @@ case class VacuumTableCommand(
} else {
getCoveringPaths(sparkSession, table.get)
}
DatabricksAtomicCommitProtocol.vacuum(sparkSession, pathsToVacuum, horizonHours)
DirectoryAtomicCommitProtocol.vacuum(sparkSession, pathsToVacuum, horizonHours)
.map(p => Row(p.toString))
}

Expand Down
114 changes: 114 additions & 0 deletions sql/core/src/main/scala/com/databricks/util/Clock.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
/*
* 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 com.databricks.util

/**
* An interface to represent clocks, so that they can be mocked out in unit tests.
*/
trait Clock {
def getTimeMillis(): Long
def waitTillTime(targetTime: Long): Long
}

/**
* A clock backed by the actual time from the OS as reported by the `System` API.
*/
class SystemClock extends Clock {

val minPollTime = 25L

/**
* @return the same time (milliseconds since the epoch)
* as is reported by `System.currentTimeMillis()`
*/
def getTimeMillis(): Long = System.currentTimeMillis()

/**
* @param targetTime block until the current time is at least this value
* @return current system time when wait has completed
*/
def waitTillTime(targetTime: Long): Long = {
var currentTime = 0L
currentTime = System.currentTimeMillis()

var waitTime = targetTime - currentTime
if (waitTime <= 0) {
return currentTime
}

val pollTime = math.max(waitTime / 10.0, minPollTime).toLong

while (true) {
currentTime = System.currentTimeMillis()
waitTime = targetTime - currentTime
if (waitTime <= 0) {
return currentTime
}
val sleepTime = math.min(waitTime, pollTime)
Thread.sleep(sleepTime)
}
-1
}
}

/**
* A `Clock` whose time can be manually set and modified. Its reported time does not change
* as time elapses, but only as its time is modified by callers. This is mainly useful for
* testing.
*
* @param time initial time (in milliseconds since the epoch)
*/
class ManualClock(private var time: Long) extends Clock {

/**
* @return `ManualClock` with initial time 0
*/
def this() = this(0L)

def getTimeMillis(): Long =
synchronized {
time
}

/**
* @param timeToSet new time (in milliseconds) that the clock should represent
*/
def setTime(timeToSet: Long): Unit = synchronized {
time = timeToSet
notifyAll()
}

/**
* @param timeToAdd time (in milliseconds) to add to the clock's time
*/
def advance(timeToAdd: Long): Unit = synchronized {
time += timeToAdd
notifyAll()
}

/**
* @param targetTime block until the clock time is set or advanced to at least this time
* @return current time reported by the clock when waiting finishes
*/
def waitTillTime(targetTime: Long): Long = synchronized {
while (time < targetTime) {
wait(10)
}
getTimeMillis()
}
}
Loading

0 comments on commit 0985111

Please sign in to comment.