Skip to content
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

Added GcsLogStore implementation #1

Merged
merged 9 commits into from
Dec 2, 2020
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/*
* Copyright (2020) The Delta Lake Project Authors.
*
* Licensed 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.delta.storage

import java.io.{IOException, _}
import java.nio.charset.StandardCharsets.UTF_8
import java.nio.file.FileAlreadyExistsException

import com.google.common.base.Throwables
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._
import org.apache.spark.SparkConf
import org.apache.spark.internal.Logging

/**
* The [[LogStore]] implementation for GCS, which uses gcs-connector to
* provide the necessary atomic and durability guarantees:
*
* 1. Atomic Visibility: Read/read-after-metadata-update/delete are strongly
* consistent for GCS.
*
* 2. Consistent Listing: GCS guarantees strong consistency for both object and
* bucket listing operations.
* https://cloud.google.com/storage/docs/consistency
*
* 3. Mutual Exclusion: Preconditions are used to handle race conditions.
*
* Regarding file creation, this implementation:
* - Opens a stream to write to GCS otherwise.
* - Throws [[FileAlreadyExistsException]] if file exists and overwrite is false.
* - Assumes file writing to be all-or-nothing, irrespective of overwrite option.
*/
class GcsLogStore(sparkConf: SparkConf, defaultHadoopConf: Configuration)
extends HadoopFileSystemLogStore(sparkConf, defaultHadoopConf) with Logging {

val preconditionFailedExceptionMessage = "412 Precondition Failed"

def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit = {
val fs = path.getFileSystem(getHadoopConfiguration)

// This is needed for the tests to throw error with local file system.
if (fs.isInstanceOf[LocalFileSystem] && !overwrite && fs.exists(path)) {
throw new FileAlreadyExistsException(path.toString)
}

try {
// If overwrite=false and path already exists, gcs-connector will throw
// org.apache.hadoop.fs.FileAlreadyExistsException after fs.create is invoked.
// This should be mapped to java.nio.file.FileAlreadyExistsException.
val stream = fs.create(path, overwrite)
try {
actions.map(_ + "\n").map(_.getBytes(UTF_8)).foreach(stream.write)
} finally {
stream.close()
}
} catch {
case e: org.apache.hadoop.fs.FileAlreadyExistsException =>
throw new FileAlreadyExistsException(path.toString).initCause(e.getCause)
// GCS uses preconditions to handle race conditions for multiple writers.
// If path gets created between fs.create and stream.close by an external
// agent or race conditions. Then this block will execute.
// Reference: https://cloud.google.com/storage/docs/generations-preconditions
case e: IOException if isPreconditionFailure(e) =>
if (!overwrite) {
throw new FileAlreadyExistsException(path.toString).initCause(e.getCause)
}
}
}

private def isPreconditionFailure(x: Throwable): Boolean = {
Throwables.getCausalChain(x)
.stream()
.filter(p => p != null)
.filter(p => p.getMessage != null)
.filter(p => p.getMessage.contains(preconditionFailedExceptionMessage))
.findFirst
.isPresent;
}

override def invalidateCache(): Unit = {}

override def isPartialWriteVisible(path: Path): Boolean = false
}
12 changes: 12 additions & 0 deletions src/test/scala/org/apache/spark/sql/delta/LogStoreSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -271,6 +271,18 @@ class HDFSLogStoreSuite extends LogStoreSuiteBase {
protected def shouldUseRenameToWriteCheckpoint: Boolean = true
}

class GcsLogStoreSuite extends LogStoreSuiteBase {

override val logStoreClassName: String = classOf[GcsLogStore].getName

testHadoopConf(
expectedErrMsg = "No FileSystem for scheme: fake",
"fs.fake.impl" -> classOf[FakeFileSystem].getName,
"fs.fake.impl.disable.cache" -> "true")

protected def shouldUseRenameToWriteCheckpoint: Boolean = false
}

class LocalLogStoreSuite extends LogStoreSuiteBase {

override val logStoreClassName: String = classOf[LocalLogStore].getName
Expand Down