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

[SHUFFLE] [WIP] Prototype: store shuffle file on external storage like S3 #34864

Closed
wants to merge 16 commits into from
Closed
Show file tree
Hide file tree
Changes from all 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
12 changes: 12 additions & 0 deletions assembly/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -268,5 +268,17 @@
</dependency>
</dependencies>
</profile>

<!-- Profile to include external shuffle storage jar file in Spark distribution -->
<profile>
<id>external-shuffle-storage</id>
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>external-shuffle-storage_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</profile>
</profiles>
</project>
55 changes: 55 additions & 0 deletions external-shuffle-storage/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
# External Shuffle Storage

This module provides support to store shuffle files on external shuffle storage like S3. It helps Dynamic
Allocation on Kubernetes. Spark driver could release idle executors without worrying about losing
shuffle data because the shuffle data is store on external shuffle storage which are different
from executors.

This module implements a new Shuffle Manager named as StarShuffleManager, and copies a lot of codes
from Spark SortShuffleManager. This is for a quick prototype. We want to use this as an example to discuss
with Spark community and get feedback. We will work with the community to remove code duplication later
and make StarShuffleManager more integrated with Spark code.

## How to Build Spark Distribution with StarShuffleManager jar File

Follow [Building Spark](https://spark.apache.org/docs/latest/building-spark.html) instructions,
with extra `-Pexternal-shuffle-storage` to generate the new shuffle implementation jar file.

Following is one command example to use `dev/make-distribution.sh` under Spark repo root directory:

```
./dev/make-distribution.sh --name spark-with-external-shuffle-storage --pip --tgz -Phive -Phive-thriftserver -Pkubernetes -Phadoop-3.2 -Phadoop-cloud -Dhadoop.version=3.2.0 -Pexternal-shuffle-storage
```

If you want to build a Spark docker image, you could unzip the Spark distribution tgz file, and run command like following:

```
./bin/docker-image-tool.sh -t spark-with-external-shuffle-storage build
```

This command creates `external-shuffle-storage_xxx.jar` file for StarShuffleManager
under `jars` directory in the generated Spark distribution. Now you could use this Spark
distribution to run your Spark application with external shuffle storage.

## How to Run Spark Application With External Shuffle Storage in Kubernetes

### Run Spark Application With S3 as External Shuffle Storage and Dynamic Allocation

Add configure to your Spark application like following (you need to adjust the values based on your environment):

```
spark.shuffle.manager=org.apache.spark.shuffle.StarShuffleManager
spark.shuffle.star.rootDir=s3://my_bucket_name/my_shuffle_folder
spark.dynamicAllocation.enabled=true
spark.dynamicAllocation.shuffleTracking.enabled=true
spark.dynamicAllocation.shuffleTracking.timeout=1
```

### How to specify AWS region for the S3 files

Add Spark config like following:

```
spark.hadoop.fs.s3a.endpoint.region=us-west-2
```

143 changes: 143 additions & 0 deletions external-shuffle-storage/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->

<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent_2.12</artifactId>
<version>3.3.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>

<artifactId>external-shuffle-storage_2.12</artifactId>
<packaging>jar</packaging>
<name>External Shuffle Storage</name>
<url>http://spark.apache.org/</url>

<properties>
<sbt.project.name>external-shuffle-storage</sbt.project.name>
<build.testJarPhase>none</build.testJarPhase>
<build.copyDependenciesPhase>package</build.copyDependenciesPhase>
<hadoop.deps.scope>provided</hadoop.deps.scope>
<hive.deps.scope>provided</hive.deps.scope>
<parquet.deps.scope>provided</parquet.deps.scope>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-math3</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
<artifactId>scalacheck_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>com.amazonaws</groupId>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should pull in spark-hadoop-cloud and so indirectly get its shaded full aws sdk. yes, it's big, but iat guarantees that it has a consistent set of its own dependencies (http client, jackson etc) and because it includes support for services like STS and s3 events, lets you add new features with guaranteed consistency of aws artifacts.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the suggestion! Yes, I was thinking to use that hadoop library as well, then did not do it due to wanting to start small with this prototype. It sounds a good idea to switch to hadoop library.

<artifactId>aws-java-sdk-s3</artifactId>
<version>1.11.975</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.testng</groupId>
<artifactId>testng</artifactId>
<version>6.14.3</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-deploy-plugin</artifactId>
<configuration>
<skip>true</skip>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-install-plugin</artifactId>
<configuration>
<skip>true</skip>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<outputDirectory>${jars.target.dir}</outputDirectory>
</configuration>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* This file is copied from Uber Remote Shuffle Service
* (https://github.com/uber/RemoteShuffleService) and modified.
*
* 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.starshuffle;

import io.netty.buffer.ByteBuf;

import java.io.IOException;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;

public class ByteBufUtils {
public static final void writeLengthAndString(ByteBuf buf, String str) {
if (str == null) {
buf.writeInt(-1);
return;
}

byte[] bytes = str.getBytes(StandardCharsets.UTF_8);
buf.writeInt(bytes.length);
buf.writeBytes(bytes);
}

public static final String readLengthAndString(ByteBuf buf) {
int length = buf.readInt();
if (length == -1) {
return null;
}

byte[] bytes = new byte[length];
buf.readBytes(bytes);
return new String(bytes, StandardCharsets.UTF_8);
}
}
Loading