Skip to content

Commit 07804ad

Browse files
committed
Moved ExternalShuffleService to the deploy package + other minor tweaks.
1 parent 4dc1f91 commit 07804ad

File tree

5 files changed

+17
-15
lines changed

5 files changed

+17
-15
lines changed

core/src/main/scala/org/apache/spark/deploy/worker/ExternalShuffleService.scala renamed to core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala

Lines changed: 14 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.deploy.worker
18+
package org.apache.spark.deploy
1919

2020
import org.apache.spark.{Logging, SparkConf, SecurityManager}
2121
import org.apache.spark.util.Utils
@@ -24,8 +24,8 @@ import org.apache.spark.network.netty.SparkTransportConf
2424
import org.apache.spark.network.sasl.SaslRpcHandler
2525
import org.apache.spark.network.server.TransportServer
2626
import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler
27-
2827
import java.util.concurrent.CountDownLatch
28+
import org.apache.spark.annotation.DeveloperApi
2929

3030
/**
3131
* Provides a server from which Executors can read shuffle files (rather than reading directly from
@@ -34,7 +34,7 @@ import java.util.concurrent.CountDownLatch
3434
*
3535
* Optionally requires SASL authentication in order to read. See [[SecurityManager]].
3636
*/
37-
private[worker]
37+
private[deploy]
3838
class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityManager)
3939
extends Logging {
4040

@@ -54,14 +54,19 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana
5454
/** Starts the external shuffle service if the user has configured us to. */
5555
def startIfEnabled() {
5656
if (enabled) {
57-
require(server == null, "Shuffle server already started")
58-
logInfo(s"Starting shuffle service on port $port with useSasl = $useSasl")
59-
server = transportContext.createServer(port)
57+
start()
6058
}
6159
}
6260

61+
/** Start the external shuffle service */
62+
def start() {
63+
require(server == null, "Shuffle server already started")
64+
logInfo(s"Starting shuffle service on port $port with useSasl = $useSasl")
65+
server = transportContext.createServer(port)
66+
}
67+
6368
def stop() {
64-
if (enabled && server != null) {
69+
if (server != null) {
6570
server.close()
6671
server = null
6772
}
@@ -86,14 +91,15 @@ object ExternalShuffleService extends Logging {
8691
// and we assume the user really wants it to be running
8792
sparkConf.set("spark.shuffle.service.enabled", "true")
8893
server = new ExternalShuffleService(sparkConf, securityManager)
94+
server.start()
8995

9096
installShutdownHook()
9197

9298
// keep running until the process is terminated
9399
barrier.await()
94100
}
95101

96-
private def installShutdownHook() = {
102+
private def installShutdownHook(): Unit = {
97103
Runtime.getRuntime.addShutdownHook(new Thread("External Shuffle Service shutdown thread") {
98104
override def run() {
99105
logInfo("Shutting down shuffle service.")

core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
3434
import org.apache.spark.{Logging, SecurityManager, SparkConf}
3535
import org.apache.spark.deploy.{Command, ExecutorDescription, ExecutorState}
3636
import org.apache.spark.deploy.DeployMessages._
37+
import org.apache.spark.deploy.ExternalShuffleService
3738
import org.apache.spark.deploy.master.{DriverState, Master}
3839
import org.apache.spark.deploy.worker.ui.WorkerWebUI
3940
import org.apache.spark.metrics.MetricsSystem

docs/job-scheduling.md

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -99,11 +99,6 @@ then set `yarn.nodemanager.aux-services.spark_shuffle.class` to
9999
`spark.shuffle.service.*` [configurations](configuration.html).
100100
4. Restart all `NodeManager`s in your cluster.
101101

102-
If you wish to use the external shuffle service in a Mesos deployment you need to launch the
103-
service on each Mesos slave that might run Spark jobs. A launcher script is provided in the `sbin/`
104-
directory: `sbin/start-shuffle-service.sh`. You can later stop the service by running
105-
`sbin/stop-shuffle-service.sh`. Additional configuration options can be passed in `SPARK_SHUFFLE_OPTS`.
106-
107102
### Resource Allocation Policy
108103

109104
At a high level, Spark should relinquish executors when they are no longer used and acquire

sbin/start-shuffle-service.sh

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -30,4 +30,4 @@ sbin="`cd "$sbin"; pwd`"
3030
. "$sbin/spark-config.sh"
3131
. "$SPARK_PREFIX/bin/load-spark-env.sh"
3232

33-
exec "$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.ExternalShuffleService 1
33+
exec "$sbin"/spark-daemon.sh start org.apache.spark.deploy.ExternalShuffleService 1

sbin/stop-shuffle-service.sh

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,4 +22,4 @@
2222
sbin="`dirname "$0"`"
2323
sbin="`cd "$sbin"; pwd`"
2424

25-
"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.ExternalShuffleService 1
25+
"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.ExternalShuffleService 1

0 commit comments

Comments
 (0)