forked from delta-io/delta
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[DELTA-OSS-EXTERNAL] Adding support for GCS
Adding support for Google Cloud Storage(GCS) as Delta Storage by introducing GcsLogStore. This PR addresses [issue delta-io#294]. File creation is an all-or-nothing approach to achieve atomicity and uses Gcs [preconditions]to avoid race conditions among multiple writers/drivers. This implementation relies on gcs-connector to provide necessary `FileSystem` implementations. This has been tested on a Google Dataproc cluster. #### GcsLogStore requirements 1. spark.delta.logStore.class=org.apache.spark.sql.delta.storage.GcsLogStore 2. Include gcs-connector in classpath. The Cloud Storage connector is automatically installed on Dataproc clusters. #### Usage ``` TABLE_LOCATION = 'gs://ranuvikram-test/test/delta-table' # Write data to table. data = spark.range(5, 10) data.write.format("delta").mode("append").save(TABLE_LOCATION) # Read data from table. df = spark.read.format("delta").load(TABLE_LOCATION) df.show() ``` : delta-io#294: https://cloud.google.com/storage/docs/generations-preconditions#_Preconditions Closes delta-io#560 Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Ranu Vikram <ranu010101@users.noreply.github.com> #22070 is resolved by tdas/o9ixtoaw. GitOrigin-RevId: 0a1ce1d4407637d7697b93a25d8fd6be3efe2f6d
- Loading branch information
1 parent
9a7338c
commit 90606ff
Showing
3 changed files
with
130 additions
and
2 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
97 changes: 97 additions & 0 deletions
97
contribs/src/main/scala/org/apache/spark/sql/delta/storage/GCSLogStore.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,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) | ||
// 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) | ||
} | ||
} | ||
} | ||
|
||
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 | ||
} |
31 changes: 31 additions & 0 deletions
31
contribs/src/test/scala/org/apache/spark/sql/delta/GCSLogStoreSuite.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,31 @@ | ||
/* | ||
* 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 | ||
|
||
import org.apache.spark.sql.delta.storage._ | ||
|
||
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 | ||
} |