forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 4
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-40039][SS] Introducing a streaming checkpoint file manager bas…
…ed on Hadoop's Abortable interface ### What changes were proposed in this pull request? Currently on S3 the checkpoint file manager (called `FileContextBasedCheckpointFileManager`) is available which is based on the rename operation. So when a file is opened for an atomic stream a temporary file will be used behind the scenes and when the stream is committed the file is renamed to its final location. But on S3 the rename operation will be a file copy so it has some serious performance implication. On Hadoop 3 there is new interface introduce called [Abortable](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/fs/Abortable.html) and S3AFileSystem has this capability. When the file is small (<= the block size) this will be a single PUT as commit and no operation if it is aborted. When the file is bigger then S3's multipart upload is used: so when the file is committed [a POST is sent](https://docs.aws.amazon.com/AmazonS3/latest/API/API_CompleteMultipartUpload.html) and when aborted [a DELETE will be sent](https://docs.aws.amazon.com/AmazonS3/latest/API/API_AbortMultipartUpload.html) (asynchronously). This avoids the file copying altogether. ### Why are the changes needed? For improving streaming performance. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? #### Unit test I have refactored the existing `CheckpointFileManagerTests` and run against a test filesystem which supports the `Abortable` interface (see `AbortableFileSystem` which is based on `RawLocalFileSystem`). This way we have a unit test. #### Integration test Moreover the same test can be run against AWS S3 by using an integration test (see `AwsS3AbortableStreamBasedCheckpointFileManagerSuite`): ``` -> S3_PATH=<..> AWS_ACCESS_KEY_ID=<..> AWS_SECRET_ACCESS_KEY=<..> AWS_SESSION_TOKEN=<..> ./build/mvn install -pl hadoop-cloud -Phadoop-cloud,hadoop-3,integration-test Discovery starting. Discovery completed in 346 milliseconds. Run starting. Expected test count is: 1 AwsS3AbortableStreamBasedCheckpointFileManagerSuite: - mkdirs, list, createAtomic, open, delete, exists CommitterBindingSuite: AbortableStreamBasedCheckpointFileManagerSuite: Run completed in 14 seconds, 407 milliseconds. Total number of tests run: 1 Suites: completed 4, aborted 0 Tests: succeeded 1, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` #### Performance test I have run a [small performance app](https://github.com/attilapiros/spark-ss-perf/blob/ab4c6004caffc38a218fa81fd5482a6cc07ca14f/src/main/scala/perf.scala) which uses a rate stream and foreach sink with an empty body. The results: ``` ➜ spark git:(SPARK-40039) ✗ ./bin/spark-submit ../spark-ss-perf/target/scala-2.12/performance-spark-ss_2.12-0.1.jar s3a://mybucket org.apache.spark.sql.execution.streaming.FileContextBasedCheckpointFileManager 2>&1 | grep "walCommit took" | awk '{print $7}' | datamash max 1 min 1 mean 1 median 1 perc:90 1 perc:95 1 perc:99 1 4143 3286 3528.6 3500 3742.8 3840 4076.04 ➜ spark git:(SPARK-40039) ✗ ./bin/spark-submit ../spark-ss-perf/target/scala-2.12/performance-spark-ss_2.12-0.1.jar s3a://mybucket org.apache.spark.internal.io.cloud.AbortableStreamBasedCheckpointFileManager 2>&1 | grep "walCommit took" | awk '{print $7}' | datamash max 1 min 1 mean 1 median 1 perc:90 1 perc:95 1 perc:99 1 3765 1447 2187.0217391304 1844.5 2867 2976.5 3437.85 ``` Closes apache#37474 from attilapiros/SPARK-40039. Authored-by: attilapiros <piros.attila.zsolt@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
- Loading branch information
1 parent
c078523
commit 7e4064c
Showing
11 changed files
with
539 additions
and
59 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
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,20 @@ | ||
--- | ||
layout: global | ||
title: Spark Hadoop3 Integration Tests | ||
--- | ||
|
||
# Running the Integration Tests | ||
|
||
As mocking of an external systems (like AWS S3) is not always perfect the unit testing should be | ||
extended with integration testing. This is why the build profile `integration-test` has been | ||
introduced here. When it is given (`-Pintegration-test`) for testing then only those tests are | ||
executed where the `org.apache.spark.internal.io.cloud.IntegrationTestSuite` tag is used. | ||
|
||
One example is `AwsS3AbortableStreamBasedCheckpointFileManagerSuite`. | ||
|
||
Integration tests will have some extra configurations for example selecting the external system to | ||
run the test against. Those configs are passed as environment variables and the existence of these | ||
variables must be checked by the test. | ||
Like for `AwsS3AbortableStreamBasedCheckpointFileManagerSuite` the S3 bucket used for testing | ||
is passed in the `S3A_PATH` and the credetinals to access AWS S3 are AWS_ACCESS_KEY_ID and | ||
AWS_SECRET_ACCESS_KEY (in addition you can define an optional AWS_SESSION_TOKEN too). |
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
95 changes: 95 additions & 0 deletions
95
.../scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.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,95 @@ | ||
/* | ||
* 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.internal.io.cloud | ||
|
||
import java.nio.file.FileAlreadyExistsException | ||
import java.util.EnumSet | ||
|
||
import scala.util.control.NonFatal | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs._ | ||
|
||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.sql.execution.streaming.AbstractFileContextBasedCheckpointFileManager | ||
import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream | ||
|
||
class AbortableStreamBasedCheckpointFileManager(path: Path, hadoopConf: Configuration) | ||
extends AbstractFileContextBasedCheckpointFileManager(path, hadoopConf) with Logging { | ||
|
||
if (!fc.hasPathCapability(path, CommonPathCapabilities.ABORTABLE_STREAM)) { | ||
throw new UnsupportedFileSystemException("AbortableStreamBasedCheckpointFileManager requires" + | ||
s" an fs (path: $path) with abortable stream support") | ||
} | ||
|
||
logInfo(s"Writing atomically to $path based on abortable stream") | ||
|
||
class AbortableStreamBasedFSDataOutputStream( | ||
fsDataOutputStream: FSDataOutputStream, | ||
fc: FileContext, | ||
path: Path, | ||
overwriteIfPossible: Boolean) extends CancellableFSDataOutputStream(fsDataOutputStream) { | ||
|
||
@volatile private var terminated = false | ||
|
||
override def cancel(): Unit = synchronized { | ||
if (terminated) return | ||
try { | ||
fsDataOutputStream.abort() | ||
fsDataOutputStream.close() | ||
} catch { | ||
case NonFatal(e) => | ||
logWarning(s"Error cancelling write to $path (stream: $fsDataOutputStream)", e) | ||
} finally { | ||
terminated = true | ||
} | ||
} | ||
|
||
override def close(): Unit = synchronized { | ||
if (terminated) return | ||
try { | ||
if (!overwriteIfPossible && fc.util().exists(path)) { | ||
fsDataOutputStream.abort() | ||
throw new FileAlreadyExistsException( | ||
s"Failed to close atomic stream $path (stream: $fsDataOutputStream) as destination already exists") | ||
} | ||
fsDataOutputStream.close() | ||
} catch { | ||
case NonFatal(e) => | ||
logWarning(s"Error closing $path (stream: $fsDataOutputStream)", e) | ||
} finally { | ||
terminated = true | ||
} | ||
} | ||
|
||
override def toString(): String = { | ||
fsDataOutputStream.toString | ||
} | ||
} | ||
|
||
override def createAtomic( | ||
path: Path, overwriteIfPossible: Boolean): CancellableFSDataOutputStream = { | ||
import CreateFlag._ | ||
val createFlag = if (overwriteIfPossible) { | ||
EnumSet.of(CREATE, OVERWRITE) | ||
} else { | ||
EnumSet.of(CREATE) | ||
} | ||
new AbortableStreamBasedFSDataOutputStream( | ||
fc.create(path, createFlag), fc, path, overwriteIfPossible) | ||
} | ||
} |
83 changes: 83 additions & 0 deletions
83
...a/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManagerSuite.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,83 @@ | ||
/* | ||
* 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.internal.io.cloud | ||
|
||
import java.io.File | ||
|
||
import scala.util.Properties | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs._ | ||
import org.apache.hadoop.fs.permission.FsPermission | ||
import org.scalatest.BeforeAndAfter | ||
|
||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.internal.io.cloud.abortable.AbortableFileSystem | ||
import org.apache.spark.sql.execution.streaming.CheckpointFileManager | ||
import org.apache.spark.sql.execution.streaming.CheckpointFileManagerTests | ||
|
||
class AbortableStreamBasedCheckpointFileManagerSuite | ||
extends CheckpointFileManagerTests with Logging { | ||
|
||
override def withTempHadoopPath(p: Path => Unit): Unit = { | ||
withTempDir { f: File => | ||
val basePath = new Path(AbortableFileSystem.ABORTABLE_FS_SCHEME, null, f.getAbsolutePath) | ||
p(basePath) | ||
} | ||
} | ||
|
||
override def checkLeakingCrcFiles(path: Path): Unit = { } | ||
|
||
override def createManager(path: Path): CheckpointFileManager = { | ||
val conf = new Configuration() | ||
conf.set(s"fs.AbstractFileSystem.${AbortableFileSystem.ABORTABLE_FS_SCHEME}.impl", | ||
"org.apache.spark.internal.io.cloud.abortable.AbstractAbortableFileSystem") | ||
new AbortableStreamBasedCheckpointFileManager(path, conf) | ||
} | ||
} | ||
|
||
@IntegrationTestSuite | ||
class AwsS3AbortableStreamBasedCheckpointFileManagerSuite | ||
extends AbortableStreamBasedCheckpointFileManagerSuite with BeforeAndAfter { | ||
|
||
val s3aPath = Properties.envOrNone("S3A_PATH") | ||
|
||
val hadoopConf = new Configuration() | ||
|
||
var cleanup: () => Unit = () => {} | ||
|
||
override protected def beforeAll(): Unit = { | ||
assert(s3aPath.isDefined, "S3A_PATH must be defined!") | ||
val path = new Path(s3aPath.get) | ||
val fc = FileContext.getFileContext(path.toUri, hadoopConf) | ||
assert(!fc.util.exists(path), s"S3A_PATH ($path) should not exists!") | ||
fc.mkdir(path, FsPermission.getDirDefault, true) | ||
cleanup = () => fc.delete(path, true) | ||
} | ||
|
||
override protected def afterAll(): Unit = { | ||
cleanup() | ||
} | ||
|
||
override def withTempHadoopPath(p: Path => Unit): Unit = { | ||
p(new Path(s3aPath.get)) | ||
} | ||
|
||
override def createManager(path: Path): CheckpointFileManager = { | ||
new AbortableStreamBasedCheckpointFileManager(path, hadoopConf) | ||
} | ||
} |
29 changes: 29 additions & 0 deletions
29
hadoop-cloud/src/test/java/org/apache/spark/internal/io/cloud/IntegrationTestSuite.java
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,29 @@ | ||
/* | ||
* 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.internal.io.cloud; | ||
|
||
import org.scalatest.TagAnnotation; | ||
|
||
import java.lang.annotation.ElementType; | ||
import java.lang.annotation.Retention; | ||
import java.lang.annotation.RetentionPolicy; | ||
import java.lang.annotation.Target; | ||
|
||
@TagAnnotation | ||
@Retention(RetentionPolicy.RUNTIME) | ||
@Target({ElementType.METHOD, ElementType.TYPE}) | ||
public @interface IntegrationTestSuite {} |
Oops, something went wrong.