Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -17,63 +17,13 @@

package org.apache.spark.network.shuffle.kubernetes;

import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.sasl.SecretKeyHolder;
import org.apache.spark.network.shuffle.ExternalShuffleClient;
import org.apache.spark.network.shuffle.protocol.RegisterDriver;
import org.apache.spark.network.util.TransportConf;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;

/**
* A client for talking to the external shuffle service in Kubernetes cluster mode.
*
* This is used by the each Spark executor to register with a corresponding external
* shuffle service on the cluster. The purpose is for cleaning up shuffle files
* reliably if the application exits unexpectedly.
*/
public class KubernetesExternalShuffleClient extends ExternalShuffleClient {
private static final Logger logger = LoggerFactory
.getLogger(KubernetesExternalShuffleClient.class);

/**
* Creates an Kubernetes external shuffle client that wraps the {@link ExternalShuffleClient}.
* Please refer to docs on {@link ExternalShuffleClient} for more information.
*/
public KubernetesExternalShuffleClient(
TransportConf conf,
SecretKeyHolder secretKeyHolder,
boolean saslEnabled) {
super(conf, secretKeyHolder, saslEnabled);
}

public void registerDriverWithShuffleService(String host, int port)
throws IOException, InterruptedException {
checkInit();
ByteBuffer registerDriver = new RegisterDriver(appId, 0).toByteBuffer();
TransportClient client = clientFactory.createClient(host, port);
client.sendRpc(registerDriver, new RegisterDriverCallback());
}

private class RegisterDriverCallback implements RpcResponseCallback {
@Override
public void onSuccess(ByteBuffer response) {
logger.info("Successfully registered app " + appId + " with external shuffle service.");
}
public interface KubernetesExternalShuffleClient extends Closeable {

@Override
public void onFailure(Throwable e) {
logger.warn("Unable to register app " + appId + " with external shuffle service. " +
"Please manually remove shuffle data after driver exit. Error: " + e);
}
}
void init(String appId);

@Override
public void close() {
super.close();
}
void registerDriverWithShuffleService(String host, int port)
throws IOException, InterruptedException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.shuffle.kubernetes;
Copy link
Author

Choose a reason for hiding this comment

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

This is equivalent to what used to be KubernetesExternalShuffleClient, with just a rename and an interface extraction. I suppose git doesn't present that in the most intuitive way here.


import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.sasl.SecretKeyHolder;
import org.apache.spark.network.shuffle.ExternalShuffleClient;
import org.apache.spark.network.shuffle.protocol.RegisterDriver;
import org.apache.spark.network.util.TransportConf;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.nio.ByteBuffer;

/**
* A client for talking to the external shuffle service in Kubernetes cluster mode.
*
* This is used by the each Spark executor to register with a corresponding external
* shuffle service on the cluster. The purpose is for cleaning up shuffle files
* reliably if the application exits unexpectedly.
*/
public class KubernetesExternalShuffleClientImpl
extends ExternalShuffleClient implements KubernetesExternalShuffleClient {

private static final Logger logger = LoggerFactory
.getLogger(KubernetesExternalShuffleClientImpl.class);

/**
* Creates a Kubernetes external shuffle client that wraps the {@link ExternalShuffleClient}.
* Please refer to docs on {@link ExternalShuffleClient} for more information.
*/
public KubernetesExternalShuffleClientImpl(
TransportConf conf,
SecretKeyHolder secretKeyHolder,
boolean saslEnabled) {
super(conf, secretKeyHolder, saslEnabled);
}

@Override
public void registerDriverWithShuffleService(String host, int port)
throws IOException, InterruptedException {
checkInit();
ByteBuffer registerDriver = new RegisterDriver(appId, 0).toByteBuffer();
TransportClient client = clientFactory.createClient(host, port);
client.sendRpc(registerDriver, new RegisterDriverCallback());
}

private class RegisterDriverCallback implements RpcResponseCallback {
@Override
public void onSuccess(ByteBuffer response) {
logger.info("Successfully registered app " + appId + " with external shuffle service.");
}

@Override
public void onFailure(Throwable e) {
logger.warn("Unable to register app " + appId + " with external shuffle service. " +
"Please manually remove shuffle data after driver exit. Error: " + e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@ private[spark] trait ExecutorPodFactory {
applicationId: String,
driverUrl: String,
executorEnvs: Seq[(String, String)],
shuffleServiceConfig: Option[ShuffleServiceConfig],
driverPod: Pod,
nodeToLocalTaskCount: Map[String, Int]): Pod
}
Expand All @@ -47,7 +46,8 @@ private[spark] class ExecutorPodFactoryImpl(
mountSecretsBootstrap: Option[MountSecretsBootstrap],
mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap],
executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap],
executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin])
executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin],
shuffleManager: Option[KubernetesExternalShuffleManager])
extends ExecutorPodFactory {

import ExecutorPodFactoryImpl._
Expand Down Expand Up @@ -111,7 +111,6 @@ private[spark] class ExecutorPodFactoryImpl(
applicationId: String,
driverUrl: String,
executorEnvs: Seq[(String, String)],
shuffleServiceConfig: Option[ShuffleServiceConfig],
driverPod: Pod,
nodeToLocalTaskCount: Map[String, Int]): Pod = {
val name = s"$executorPodNamePrefix-exec-$executorId"
Expand Down Expand Up @@ -179,6 +178,9 @@ private[spark] class ExecutorPodFactoryImpl(
.withContainerPort(port._2)
.build()
})
val shuffleVolumesWithMounts =
shuffleManager.map(_.getExecutorShuffleDirVolumesWithMounts)
.getOrElse(Seq.empty)

val executorContainer = new ContainerBuilder()
.withName(s"executor")
Expand All @@ -191,6 +193,7 @@ private[spark] class ExecutorPodFactoryImpl(
.endResources()
.addAllToEnv(executorEnv.asJava)
.withPorts(requiredPorts.asJava)
.addAllToVolumeMounts(shuffleVolumesWithMounts.map(_._2).asJava)
.build()

val executorPod = new PodBuilder()
Expand All @@ -211,6 +214,7 @@ private[spark] class ExecutorPodFactoryImpl(
.withHostname(hostname)
.withRestartPolicy("Never")
.withNodeSelector(nodeSelector.asJava)
.addAllToVolumes(shuffleVolumesWithMounts.map(_._1).asJava)
.endSpec()
.build()

Expand All @@ -226,42 +230,15 @@ private[spark] class ExecutorPodFactoryImpl(
.build()
}.getOrElse(executorContainer)

val withMaybeShuffleConfigExecutorContainer = shuffleServiceConfig.map { config =>
config.shuffleDirs.foldLeft(containerWithExecutorLimitCores) { (container, dir) =>
new ContainerBuilder(container)
.addNewVolumeMount()
.withName(FilenameUtils.getBaseName(dir))
.withMountPath(dir)
.endVolumeMount()
.build()
}
}.getOrElse(containerWithExecutorLimitCores)
val withMaybeShuffleConfigPod = shuffleServiceConfig.map { config =>
config.shuffleDirs.foldLeft(executorPod) { (builder, dir) =>
new PodBuilder(builder)
.editSpec()
.addNewVolume()
.withName(FilenameUtils.getBaseName(dir))
.withNewHostPath()
.withPath(dir)
.endHostPath()
.endVolume()
.endSpec()
.build()
}
}.getOrElse(executorPod)

val (withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) =
mountSecretsBootstrap.map {bootstrap =>
bootstrap.mountSecrets(withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)
}.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer))

bootstrap.mountSecrets(executorPod, containerWithExecutorLimitCores)
}.getOrElse((executorPod, containerWithExecutorLimitCores))
val (withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer) =
mountSmallFilesBootstrap.map { bootstrap =>
bootstrap.mountSmallFilesSecret(
withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer)
}.getOrElse((withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer))

val (executorPodWithInitContainer, initBootstrappedExecutorContainer) =
executorInitContainerBootstrap.map { bootstrap =>
val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,10 @@ import org.apache.spark.deploy.kubernetes.config._
import org.apache.spark.deploy.kubernetes.constants._
import org.apache.spark.deploy.kubernetes.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrapImpl}
import org.apache.spark.internal.Logging
import org.apache.spark.network.netty.SparkTransportConf
import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClientImpl
import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
import org.apache.spark.util.Utils

private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging {

Expand Down Expand Up @@ -109,17 +112,31 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
sparkConf,
Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)),
Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH)))

val kubernetesShuffleManager = if (Utils.isDynamicAllocationEnabled(sparkConf)) {
val kubernetesExternalShuffleClient = new KubernetesExternalShuffleClientImpl(
Copy link
Member

Choose a reason for hiding this comment

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

Does this ensure that each executor gets a different instance of the shuffle client?

Copy link
Author

Choose a reason for hiding this comment

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

KubernetesClusterManager is on the driver.

SparkTransportConf.fromSparkConf(sparkConf, "shuffle"),
sc.env.securityManager,
sc.env.securityManager.isAuthenticationEnabled())
Some(new KubernetesExternalShuffleManagerImpl(
sparkConf,
kubernetesClient,
kubernetesExternalShuffleClient))
} else None

val executorPodFactory = new ExecutorPodFactoryImpl(
sparkConf,
NodeAffinityExecutorPodModifierImpl,
mountSecretBootstrap,
mountSmallFilesBootstrap,
executorInitContainerBootstrap,
executorInitContainerSecretVolumePlugin)
executorInitContainerSecretVolumePlugin,
kubernetesShuffleManager)
new KubernetesClusterSchedulerBackend(
sc.taskScheduler.asInstanceOf[TaskSchedulerImpl],
sc,
executorPodFactory,
kubernetesShuffleManager,
kubernetesClient)
}

Expand Down
Loading