Skip to content

[SPARK-3797] Run external shuffle service in Yarn NM #3082

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

Closed
wants to merge 23 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
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
Expand Up @@ -66,7 +66,6 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
// Lower and upper bounds on the number of executors. These are required.
private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
verifyBounds()

// How long there must be backlogged tasks for before an addition is triggered
private val schedulerBacklogTimeout = conf.getLong(
Expand All @@ -77,9 +76,14 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
"spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)

// How long an executor must be idle for before it is removed
private val removeThresholdSeconds = conf.getLong(
private val executorIdleTimeout = conf.getLong(
"spark.dynamicAllocation.executorIdleTimeout", 600)

// During testing, the methods to actually kill and add executors are mocked out
private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false)
Copy link
Contributor

Choose a reason for hiding this comment

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

"spark.test.dynamicAllocation" might be a name more in line with other test-only properties. Also please add some documentation or name that indicates what it means to be testing.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This was actually introduced in a different patch and I'm just moving it around. Also, by testing I mean when I'm mocking its behavior in the relevant test suite. I'm not sure what else I can add to that.

Copy link
Contributor

Choose a reason for hiding this comment

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

Gotcha. That's sufficient, I think -- "During testing, the methods to actually kill and add executors are mocked out."


validateSettings()

// Number of executors to add in the next round
private var numExecutorsToAdd = 1

Expand All @@ -103,17 +107,14 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
// Polling loop interval (ms)
private val intervalMillis: Long = 100

// Whether we are testing this class. This should only be used internally.
private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false)

// Clock used to schedule when executors should be added and removed
private var clock: Clock = new RealClock

/**
* Verify that the lower and upper bounds on the number of executors are valid.
* Verify that the settings specified through the config are valid.
* If not, throw an appropriate exception.
*/
private def verifyBounds(): Unit = {
private def validateSettings(): Unit = {
if (minNumExecutors < 0 || maxNumExecutors < 0) {
throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
}
Expand All @@ -124,6 +125,22 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " +
s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!")
}
if (schedulerBacklogTimeout <= 0) {
throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!")
}
if (sustainedSchedulerBacklogTimeout <= 0) {
throw new SparkException(
"spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!")
}
if (executorIdleTimeout <= 0) {
throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be > 0!")
}
// Require external shuffle service for dynamic allocation
Copy link
Contributor

Choose a reason for hiding this comment

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

Dynamic allocation, at the very least in the sense of dynamically adding executors, can still be useful without the external shuffle service. I hadn't noticed before that dynamic allocation will kill containers with blocks on them. Even with the external shuffle service, isn't this a problem for cached blocks? Would you be opposed to me adding a mode that avoids this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I personally don't see much use of dynamic allocation if you can't both add and remove executors. By default we start the cluster at the max number of executors anyway, so if we don't remove executors there's little point in adding them.

Now, if we do support both, then we have to enable external shuffle service because it's basically totally broken if we kill executors without it. Shuffle is different from caching in that it's not only an optimization to keep the shuffle files but also a necessity for correctness. Since shuffling is such a common thing in Spark, I think we should fail the application early before the user realizes his/her job is being re-run over and over again.

But yes, I think we left out the caching story so far. My original design is to add a warning or maybe throw an exception if this is enabled and the user tries to cache stuff, and I believe we still need to add that.

Copy link
Contributor

Choose a reason for hiding this comment

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

Even without the external shuffle service, there are opportunities to remove executors, no? Does the shuffle service keep data after jobs have completed?

Copy link
Contributor

Choose a reason for hiding this comment

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

(Either way, doesn't need to hold up this JIRA. Given the current implementation, disabling dynamic allocation when there's no external shuffle service seems like the right thing to do.)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hm, that's true. We can still call sc.killExecutors without the external shuffle service, though if the application explicitly calls that then I would assume that it's aware of what it's doing. In this case I prefer to require the service for dynamic allocation because much of this happens under the hood.

// Otherwise, we may lose shuffle files when killing executors
if (!conf.getBoolean("spark.shuffle.service.enabled", false) && !testing) {
throw new SparkException("Dynamic allocation of executors requires the external " +
"shuffle service. You may enable this through spark.shuffle.service.enabled.")
}
}

/**
Expand Down Expand Up @@ -254,7 +271,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
val removeRequestAcknowledged = testing || sc.killExecutor(executorId)
if (removeRequestAcknowledged) {
logInfo(s"Removing executor $executorId because it has been idle for " +
s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})")
s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})")
executorsPendingToRemove.add(executorId)
true
} else {
Expand Down Expand Up @@ -329,8 +346,8 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
private def onExecutorIdle(executorId: String): Unit = synchronized {
if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) {
logDebug(s"Starting idle timer for $executorId because there are no more tasks " +
s"scheduled to run on the executor (to expire in $removeThresholdSeconds seconds)")
removeTimes(executorId) = clock.getTimeMillis + removeThresholdSeconds * 1000
s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)")
removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,6 @@ import org.apache.spark.network.util.{ConfigProvider, TransportConf}
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.ShuffleManager
import org.apache.spark.shuffle.hash.HashShuffleManager
import org.apache.spark.shuffle.sort.SortShuffleManager
import org.apache.spark.util._

private[spark] sealed trait BlockValues
Expand Down Expand Up @@ -97,7 +96,12 @@ private[spark] class BlockManager(

private[spark]
val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false)
private val externalShuffleServicePort = conf.getInt("spark.shuffle.service.port", 7337)

// Port used by the external shuffle service. In Yarn mode, this may be already be
// set through the Hadoop configuration as the server is launched in the Yarn NM.
private val externalShuffleServicePort =
Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt

// Check that we're not using external shuffle service with consolidated shuffle files.
if (externalShuffleServiceEnabled
&& conf.getBoolean("spark.shuffle.consolidateFiles", false)
Expand Down
16 changes: 16 additions & 0 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ import org.json4s._
import tachyon.client.{TachyonFile,TachyonFS}

import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}

/** CallSite represents a place in user code. It can have a short and a long form. */
Expand Down Expand Up @@ -1780,6 +1781,21 @@ private[spark] object Utils extends Logging {
val manifest = new JarManifest(manifestUrl.openStream())
manifest.getMainAttributes.getValue(Name.IMPLEMENTATION_VERSION)
}.getOrElse("Unknown")

/**
* Return the value of a config either through the SparkConf or the Hadoop configuration
* if this is Yarn mode. In the latter case, this defaults to the value set through SparkConf
* if the key is not set in the Hadoop configuration.
*/
def getSparkOrYarnConfig(conf: SparkConf, key: String, default: String): String = {
val sparkValue = conf.get(key, default)
if (SparkHadoopUtil.get.isYarnMode) {
SparkHadoopUtil.get.newConfiguration(conf).get(key, sparkValue)
} else {
sparkValue
}
}

}

/**
Expand Down
3 changes: 3 additions & 0 deletions make-distribution.sh
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,9 @@ echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DI
# Copy jars
cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
cp "$FWDIR"/network/yarn/target/scala*/spark-network-yarn*.jar "$DISTDIR/lib/"
Copy link
Contributor

Choose a reason for hiding this comment

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

I wonder if it would be nicer to use maven-shade-plugin to create a single jar for the NM aux service. That might make it easier for people to install it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes I plan to do that though in a separate PR. See my comment in andrewor14@f39daa6

Copy link
Contributor

Choose a reason for hiding this comment

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

@andrewor14
Here is a problem:
I use this command line: ./make-distribution.sh -Dhadoop.version=2.3.0-cdh5.0.1 -Dyarn.version=2.3.0-cdh5.0.1 -Phadoop-2.3 -Pyarn -Pnetlib-lgpl , but $FWDIR"/network/yarn/target/scala*/spark-network-yarn*.jar does not exist

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes I will fix this in a separate PR

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hey @witgo I just pushed a hot fix. I didn't realize make-distribution.sh fails fast if a copy fails. Thanks for bringing this up.

cp "$FWDIR"/network/yarn/target/scala*/spark-network-shuffle*.jar "$DISTDIR/lib/"
cp "$FWDIR"/network/yarn/target/scala*/spark-network-common*.jar "$DISTDIR/lib/"

# Copy example sources (needed for python and SQL)
mkdir -p "$DISTDIR/examples/src/main"
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
* 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.network.sasl;

import java.lang.Override;
import java.nio.ByteBuffer;
import java.nio.charset.Charset;
import java.util.concurrent.ConcurrentHashMap;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.spark.network.sasl.SecretKeyHolder;

/**
* A class that manages shuffle secret used by the external shuffle service.
*/
public class ShuffleSecretManager implements SecretKeyHolder {
private final Logger logger = LoggerFactory.getLogger(ShuffleSecretManager.class);
private final ConcurrentHashMap<String, String> shuffleSecretMap;

private static final Charset UTF8_CHARSET = Charset.forName("UTF-8");

// Spark user used for authenticating SASL connections
// Note that this must match the value in org.apache.spark.SecurityManager
private static final String SPARK_SASL_USER = "sparkSaslUser";

/**
* Convert the given string to a byte buffer. The resulting buffer can be converted back to
* the same string through {@link #bytesToString(ByteBuffer)}. This is used if the external
* shuffle service represents shuffle secrets as bytes buffers instead of strings.
*/
public static ByteBuffer stringToBytes(String s) {
return ByteBuffer.wrap(s.getBytes(UTF8_CHARSET));
}

/**
* Convert the given byte buffer to a string. The resulting string can be converted back to
* the same byte buffer through {@link #stringToBytes(String)}. This is used if the external
* shuffle service represents shuffle secrets as bytes buffers instead of strings.
*/
public static String bytesToString(ByteBuffer b) {
return new String(b.array(), UTF8_CHARSET);
}

public ShuffleSecretManager() {
shuffleSecretMap = new ConcurrentHashMap<String, String>();
}

/**
* Register an application with its secret.
* Executors need to first authenticate themselves with the same secret before
* fetching shuffle files written by other executors in this application.
*/
public void registerApp(String appId, String shuffleSecret) {
if (!shuffleSecretMap.contains(appId)) {
shuffleSecretMap.put(appId, shuffleSecret);
logger.info("Registered shuffle secret for application {}", appId);
} else {
logger.debug("Application {} already registered", appId);
}
}

/**
* Register an application with its secret specified as a byte buffer.
*/
public void registerApp(String appId, ByteBuffer shuffleSecret) {
registerApp(appId, bytesToString(shuffleSecret));
}

/**
* Unregister an application along with its secret.
* This is called when the application terminates.
*/
public void unregisterApp(String appId) {
if (shuffleSecretMap.contains(appId)) {
shuffleSecretMap.remove(appId);
logger.info("Unregistered shuffle secret for application {}", appId);
} else {
logger.warn("Attempted to unregister application {} when it is not registered", appId);
}
}

/**
* Return the Spark user for authenticating SASL connections.
*/
@Override
public String getSaslUser(String appId) {
return SPARK_SASL_USER;
}

/**
* Return the secret key registered with the given application.
* This key is used to authenticate the executors before they can fetch shuffle files
* written by this application from the external shuffle service. If the specified
* application is not registered, return null.
*/
@Override
public String getSecretKey(String appId) {
return shuffleSecretMap.get(appId);
}
}
58 changes: 58 additions & 0 deletions network/yarn/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
<?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</artifactId>
<version>1.2.0-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>

<groupId>org.apache.spark</groupId>
<artifactId>spark-network-yarn_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Yarn Shuffle Service Code</name>
<url>http://spark.apache.org/</url>
<properties>
<sbt.project.name>network-yarn</sbt.project.name>
</properties>

<dependencies>
<!-- Core dependencies -->
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-network-shuffle_2.10</artifactId>
<version>${project.version}</version>
</dependency>

<!-- Provided dependencies -->
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>

<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
</build>
</project>
Loading