From f6fdd6a40b38b413cd55e2315dd9fb5bc6ca575c Mon Sep 17 00:00:00 2001 From: foxish Date: Thu, 14 Sep 2017 20:10:24 -0700 Subject: [PATCH 01/29] Spark on Kubernetes - basic scheduler backend --- pom.xml | 7 + resource-managers/kubernetes/core/pom.xml | 102 ++++ .../spark/deploy/k8s/ConfigurationUtils.scala | 37 ++ .../spark/deploy/k8s/OptionRequirements.scala | 40 ++ .../k8s/SparkKubernetesClientFactory.scala | 103 ++++ .../org/apache/spark/deploy/k8s/config.scala | 157 ++++++ .../apache/spark/deploy/k8s/constants.scala | 77 +++ .../cluster/k8s/ExecutorPodFactory.scala | 228 +++++++++ .../k8s/KubernetesClusterManager.scala | 69 +++ .../KubernetesClusterSchedulerBackend.scala | 456 ++++++++++++++++++ .../core/src/test/resources/log4j.properties | 31 ++ .../cluster/k8s/ExecutorPodFactorySuite.scala | 145 ++++++ ...bernetesClusterSchedulerBackendSuite.scala | 378 +++++++++++++++ 13 files changed, 1830 insertions(+) create mode 100644 resource-managers/kubernetes/core/pom.xml create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala create mode 100644 resource-managers/kubernetes/core/src/test/resources/log4j.properties create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala diff --git a/pom.xml b/pom.xml index 9fac8b1e53788..fa4888d00f2ac 100644 --- a/pom.xml +++ b/pom.xml @@ -2648,6 +2648,13 @@ + + kubernetes + + resource-managers/kubernetes/core + + + hive-thriftserver diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml new file mode 100644 index 0000000000000..1637c0f7aa716 --- /dev/null +++ b/resource-managers/kubernetes/core/pom.xml @@ -0,0 +1,102 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes_2.11 + jar + Spark Project Kubernetes + + kubernetes + 2.2.13 + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + + io.fabric8 + kubernetes-client + ${kubernetes.client.version} + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + + + + + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + ${fasterxml.jackson.version} + + + + + com.google.guava + guava + + + + + org.mockito + mockito-core + test + + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala new file mode 100644 index 0000000000000..eb0e4328acae0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -0,0 +1,37 @@ +/* + * 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.deploy.k8s + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging + +private[spark] object ConfigurationUtils extends Logging { + def parsePrefixedKeyValuePairs( + sparkConf: SparkConf, + prefix: String, + configType: String): Map[String, String] = { + val fromPrefix = sparkConf.getAllWithPrefix(prefix) + fromPrefix.groupBy(_._1).foreach { + case (key, values) => + require(values.size == 1, + s"Cannot have multiple values for a given $configType key, got key $key with" + + s" values $values") + } + fromPrefix.toMap + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala new file mode 100644 index 0000000000000..89053de5b9558 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala @@ -0,0 +1,40 @@ +/* + * 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.deploy.k8s + +private[spark] object OptionRequirements { + + def requireBothOrNeitherDefined( + opt1: Option[_], + opt2: Option[_], + errMessageWhenFirstIsMissing: String, + errMessageWhenSecondIsMissing: String): Unit = { + requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) + requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) + } + + def requireSecondIfFirstIsDefined( + opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { + opt1.foreach { _ => + require(opt2.isDefined, errMessageWhenSecondIsMissing) + } + } + + def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { + opt1.foreach { _ => require(opt2.isEmpty, errMessage) } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala new file mode 100644 index 0000000000000..c90303b6ba506 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -0,0 +1,103 @@ +/* + * 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.deploy.k8s + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient} +import io.fabric8.kubernetes.client.utils.HttpClientUtils +import okhttp3.Dispatcher + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.util.ThreadUtils + +/** + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL + * options for different components. + */ +private[spark] object SparkKubernetesClientFactory { + + def createKubernetesClient( + master: String, + namespace: Option[String], + kubernetesAuthConfPrefix: String, + sparkConf: SparkConf, + maybeServiceAccountToken: Option[File], + maybeServiceAccountCaCert: Option[File]): KubernetesClient = { + val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX" + val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX" + val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf) + .map(new File(_)) + .orElse(maybeServiceAccountToken) + val oauthTokenValue = sparkConf.getOption(oauthTokenConf) + OptionRequirements.requireNandDefined( + oauthTokenFile, + oauthTokenValue, + s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" + + s" value $oauthTokenConf.") + + val caCertFile = sparkConf + .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX") + .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath)) + val clientKeyFile = sparkConf + .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") + val clientCertFile = sparkConf + .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") + val dispatcher = new Dispatcher( + ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) + val config = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(master) + .withWebsocketPingInterval(0) + .withOption(oauthTokenValue) { + (token, configBuilder) => configBuilder.withOauthToken(token) + }.withOption(oauthTokenFile) { + (file, configBuilder) => + configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8)) + }.withOption(caCertFile) { + (file, configBuilder) => configBuilder.withCaCertFile(file) + }.withOption(clientKeyFile) { + (file, configBuilder) => configBuilder.withClientKeyFile(file) + }.withOption(clientCertFile) { + (file, configBuilder) => configBuilder.withClientCertFile(file) + }.withOption(namespace) { + (ns, configBuilder) => configBuilder.withNamespace(ns) + }.build() + val baseHttpClient = HttpClientUtils.createHttpClient(config) + val httpClientWithCustomDispatcher = baseHttpClient.newBuilder() + .dispatcher(dispatcher) + .build() + new DefaultKubernetesClient(httpClientWithCustomDispatcher, config) + } + + private implicit class OptionConfigurableConfigBuilder(configBuilder: ConfigBuilder) { + + def withOption[T] + (option: Option[T]) + (configurator: ((T, ConfigBuilder) => ConfigBuilder)): OptionConfigurableConfigBuilder = { + new OptionConfigurableConfigBuilder(option.map { opt => + configurator(opt, configBuilder) + }.getOrElse(configBuilder)) + } + + def build(): Config = configBuilder.build() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala new file mode 100644 index 0000000000000..e05b9d99c962f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala @@ -0,0 +1,157 @@ +/* + * 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.deploy.k8s + +import org.apache.spark.{SPARK_VERSION => sparkVersion} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.network.util.ByteUnit + +package object config extends Logging { + + private[spark] val KUBERNETES_NAMESPACE = + ConfigBuilder("spark.kubernetes.namespace") + .doc("The namespace that will be used for running the driver and executor pods. When using" + + " spark-submit in cluster mode, this can also be passed to spark-submit via the" + + " --kubernetes-namespace command line argument.") + .stringConf + .createWithDefault("default") + + private[spark] val DRIVER_DOCKER_IMAGE = + ConfigBuilder("spark.kubernetes.driver.docker.image") + .doc("Docker image to use for the driver. Specify this using the standard Docker tag format.") + .stringConf + .createWithDefault(s"spark-driver:$sparkVersion") + + private[spark] val EXECUTOR_DOCKER_IMAGE = + ConfigBuilder("spark.kubernetes.executor.docker.image") + .doc("Docker image to use for the executors. Specify this using the standard Docker tag" + + " format.") + .stringConf + .createWithDefault(s"spark-executor:$sparkVersion") + + private[spark] val DOCKER_IMAGE_PULL_POLICY = + ConfigBuilder("spark.kubernetes.docker.image.pullPolicy") + .doc("Docker image pull policy when pulling any docker image in Kubernetes integration") + .stringConf + .createWithDefault("IfNotPresent") + + + private[spark] val APISERVER_AUTH_DRIVER_CONF_PREFIX = + "spark.kubernetes.authenticate.driver" + private[spark] val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX = + "spark.kubernetes.authenticate.driver.mounted" + private[spark] val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken" + private[spark] val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile" + private[spark] val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile" + private[spark] val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile" + private[spark] val CA_CERT_FILE_CONF_SUFFIX = "caCertFile" + + private[spark] val KUBERNETES_SERVICE_ACCOUNT_NAME = + ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName") + .doc("Service account that is used when running the driver pod. The driver pod uses" + + " this service account when requesting executor pods from the API server. If specific" + + " credentials are given for the driver pod to use, the driver will favor" + + " using those credentials instead.") + .stringConf + .createOptional + + // Note that while we set a default for this when we start up the + // scheduler, the specific default value is dynamically determined + // based on the executor memory. + private[spark] val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD = + ConfigBuilder("spark.kubernetes.executor.memoryOverhead") + .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" + + " is memory that accounts for things like VM overheads, interned strings, other native" + + " overheads, etc. This tends to grow with the executor size. (typically 6-10%).") + .bytesConf(ByteUnit.MiB) + .createOptional + + private[spark] val KUBERNETES_DRIVER_MEMORY_OVERHEAD = + ConfigBuilder("spark.kubernetes.driver.memoryOverhead") + .doc("The amount of off-heap memory (in megabytes) to be allocated for the driver and the" + + " driver submission server. This is memory that accounts for things like VM overheads," + + " interned strings, other native overheads, etc. This tends to grow with the driver's" + + " memory size (typically 6-10%).") + .bytesConf(ByteUnit.MiB) + .createOptional + + private[spark] val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label." + private[spark] val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation." + private[spark] val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." + private[spark] val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." + private[spark] val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv." + private[spark] val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets." + private[spark] val KUBERNETES_EXECUTOR_SECRETS_PREFIX = "spark.kubernetes.executor.secrets." + + private[spark] val KUBERNETES_DRIVER_POD_NAME = + ConfigBuilder("spark.kubernetes.driver.pod.name") + .doc("Name of the driver pod.") + .stringConf + .createOptional + + private[spark] val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = + ConfigBuilder("spark.kubernetes.executor.podNamePrefix") + .doc("Prefix to use in front of the executor pod names.") + .internal() + .stringConf + .createWithDefault("spark") + + private[spark] val KUBERNETES_ALLOCATION_BATCH_SIZE = + ConfigBuilder("spark.kubernetes.allocation.batch.size") + .doc("Number of pods to launch at once in each round of dynamic allocation. ") + .intConf + .createWithDefault(5) + + private[spark] val KUBERNETES_ALLOCATION_BATCH_DELAY = + ConfigBuilder("spark.kubernetes.allocation.batch.delay") + .doc("Number of seconds to wait between each round of executor allocation. ") + .longConf + .createWithDefault(1) + + private[spark] val INIT_CONTAINER_JARS_DOWNLOAD_LOCATION = + ConfigBuilder("spark.kubernetes.mountdependencies.jarsDownloadDir") + .doc("Location to download jars to in the driver and executors. When using" + + " spark-submit, this directory must be empty and will be mounted as an empty directory" + + " volume on the driver and executor pod.") + .stringConf + .createWithDefault("/var/spark-data/spark-jars") + + private[spark] val KUBERNETES_EXECUTOR_LIMIT_CORES = + ConfigBuilder("spark.kubernetes.executor.limit.cores") + .doc("Specify the hard cpu limit for a single executor pod") + .stringConf + .createOptional + + private[spark] val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." + + private[spark] def resolveK8sMaster(rawMasterString: String): String = { + if (!rawMasterString.startsWith("k8s://")) { + throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") + } + val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "") + if (masterWithoutK8sPrefix.startsWith("http://") + || masterWithoutK8sPrefix.startsWith("https://")) { + masterWithoutK8sPrefix + } else { + val resolvedURL = s"https://$masterWithoutK8sPrefix" + logDebug(s"No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + + s" URL is $resolvedURL") + resolvedURL + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala new file mode 100644 index 0000000000000..611afe4883854 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala @@ -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.deploy.k8s + +package object constants { + // Labels + private[spark] val SPARK_APP_ID_LABEL = "spark-app-selector" + private[spark] val SPARK_EXECUTOR_ID_LABEL = "spark-exec-id" + private[spark] val SPARK_ROLE_LABEL = "spark-role" + private[spark] val SPARK_POD_DRIVER_ROLE = "driver" + private[spark] val SPARK_POD_EXECUTOR_ROLE = "executor" + + // Credentials secrets + private[spark] val DRIVER_CREDENTIALS_SECRETS_BASE_DIR = + "/mnt/secrets/spark-kubernetes-credentials" + private[spark] val DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME = "ca-cert" + private[spark] val DRIVER_CREDENTIALS_CA_CERT_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME" + private[spark] val DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME = "client-key" + private[spark] val DRIVER_CREDENTIALS_CLIENT_KEY_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME" + private[spark] val DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME = "client-cert" + private[spark] val DRIVER_CREDENTIALS_CLIENT_CERT_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME" + private[spark] val DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME = "oauth-token" + private[spark] val DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME" + private[spark] val DRIVER_CREDENTIALS_SECRET_VOLUME_NAME = "kubernetes-credentials" + + // Default and fixed ports + private[spark] val DEFAULT_DRIVER_PORT = 7078 + private[spark] val DEFAULT_BLOCKMANAGER_PORT = 7079 + private[spark] val DEFAULT_UI_PORT = 4040 + private[spark] val BLOCK_MANAGER_PORT_NAME = "blockmanager" + private[spark] val DRIVER_PORT_NAME = "driver-rpc-port" + private[spark] val EXECUTOR_PORT_NAME = "executor" + + // Environment Variables + private[spark] val ENV_EXECUTOR_PORT = "SPARK_EXECUTOR_PORT" + private[spark] val ENV_DRIVER_URL = "SPARK_DRIVER_URL" + private[spark] val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES" + private[spark] val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY" + private[spark] val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID" + private[spark] val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID" + private[spark] val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP" + private[spark] val ENV_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY" + private[spark] val ENV_SUBMIT_EXTRA_CLASSPATH = "SPARK_SUBMIT_EXTRA_CLASSPATH" + private[spark] val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH" + private[spark] val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH" + private[spark] val ENV_DRIVER_MAIN_CLASS = "SPARK_DRIVER_CLASS" + private[spark] val ENV_DRIVER_ARGS = "SPARK_DRIVER_ARGS" + private[spark] val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS" + private[spark] val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR" + private[spark] val ENV_PYSPARK_FILES = "PYSPARK_FILES" + private[spark] val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" + private[spark] val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_" + + // Miscellaneous + private[spark] val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" + private[spark] val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" + private[spark] val MEMORY_OVERHEAD_FACTOR = 0.10 + private[spark] val MEMORY_OVERHEAD_MIN_MIB = 384L +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala new file mode 100644 index 0000000000000..9870d1cd61d1b --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -0,0 +1,228 @@ +/* + * 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.scheduler.cluster.k8s + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} +import org.apache.commons.io.FilenameUtils + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.util.Utils + +// Configures executor pods. Construct one of these with a SparkConf to set up properties that are +// common across all executors. Then, pass in dynamic parameters into createExecutorPod. +private[spark] trait ExecutorPodFactory { + def createExecutorPod( + executorId: String, + applicationId: String, + driverUrl: String, + executorEnvs: Seq[(String, String)], + driverPod: Pod, + nodeToLocalTaskCount: Map[String, Int]): Pod +} + +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) + extends ExecutorPodFactory { + + import ExecutorPodFactoryImpl._ + + private val executorExtraClasspath = sparkConf.get( + org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) + private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) + + private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs ( + sparkConf, + KUBERNETES_EXECUTOR_LABEL_PREFIX, + "executor label") + require( + !executorLabels.contains(SPARK_APP_ID_LABEL), + s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.") + require( + !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL), + s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + + s" Spark.") + + private val executorAnnotations = + ConfigurationUtils.parsePrefixedKeyValuePairs ( + sparkConf, + KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, + "executor annotation") + private val nodeSelector = + ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, + KUBERNETES_NODE_SELECTOR_PREFIX, + "node selector") + + private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE) + private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) + private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) + private val blockmanagerPort = sparkConf + .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) + private val kubernetesDriverPodName = sparkConf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(throw new SparkException("Must specify the driver pod name")) + + private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) + + private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY) + private val executorMemoryString = sparkConf.get( + org.apache.spark.internal.config.EXECUTOR_MEMORY.key, + org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString) + + private val memoryOverheadMiB = sparkConf + .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt, + MEMORY_OVERHEAD_MIN_MIB)) + private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB + + private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d) + private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key) + + override def createExecutorPod( + executorId: String, + applicationId: String, + driverUrl: String, + executorEnvs: Seq[(String, String)], + driverPod: Pod, + nodeToLocalTaskCount: Map[String, Int]): Pod = { + val name = s"$executorPodNamePrefix-exec-$executorId" + + // hostname must be no longer than 63 characters, so take the last 63 characters of the pod + // name as the hostname. This preserves uniqueness since the end of name contains + // executorId and applicationId + val hostname = name.substring(Math.max(0, name.length - 63)) + val resolvedExecutorLabels = Map( + SPARK_EXECUTOR_ID_LABEL -> executorId, + SPARK_APP_ID_LABEL -> applicationId, + SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ + executorLabels + val executorMemoryQuantity = new QuantityBuilder(false) + .withAmount(s"${executorMemoryMiB}Mi") + .build() + val executorMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(s"${executorMemoryWithOverhead}Mi") + .build() + val executorCpuQuantity = new QuantityBuilder(false) + .withAmount(executorCores.toString) + .build() + val executorExtraClasspathEnv = executorExtraClasspath.map { cp => + new EnvVarBuilder() + .withName(ENV_EXECUTOR_EXTRA_CLASSPATH) + .withValue(cp) + .build() + } + val executorExtraJavaOptionsEnv = sparkConf + .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) + .map { opts => + val delimitedOpts = Utils.splitCommandString(opts) + delimitedOpts.zipWithIndex.map { + case (opt, index) => + new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() + } + }.getOrElse(Seq.empty[EnvVar]) + val executorEnv = (Seq( + (ENV_EXECUTOR_PORT, executorPort.toString), + (ENV_DRIVER_URL, driverUrl), + // Executor backend expects integral value for executor cores, so round it up to an int. + (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString), + (ENV_EXECUTOR_MEMORY, executorMemoryString), + (ENV_APPLICATION_ID, applicationId), + (ENV_EXECUTOR_ID, executorId), + (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs) + .map(env => new EnvVarBuilder() + .withName(env._1) + .withValue(env._2) + .build() + ) ++ Seq( + new EnvVarBuilder() + .withName(ENV_EXECUTOR_POD_IP) + .withValueFrom(new EnvVarSourceBuilder() + .withNewFieldRef("v1", "status.podIP") + .build()) + .build() + ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq + val requiredPorts = Seq( + (EXECUTOR_PORT_NAME, executorPort), + (BLOCK_MANAGER_PORT_NAME, blockmanagerPort)) + .map(port => { + new ContainerPortBuilder() + .withName(port._1) + .withContainerPort(port._2) + .build() + }) + + val executorContainer = new ContainerBuilder() + .withName(s"executor") + .withImage(executorDockerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .withNewResources() + .addToRequests("memory", executorMemoryQuantity) + .addToLimits("memory", executorMemoryLimitQuantity) + .addToRequests("cpu", executorCpuQuantity) + .endResources() + .addAllToEnv(executorEnv.asJava) + .withPorts(requiredPorts.asJava) + .build() + + val executorPod = new PodBuilder() + .withNewMetadata() + .withName(name) + .withLabels(resolvedExecutorLabels.asJava) + .withAnnotations(executorAnnotations.asJava) + .withOwnerReferences() + .addNewOwnerReference() + .withController(true) + .withApiVersion(driverPod.getApiVersion) + .withKind(driverPod.getKind) + .withName(driverPod.getMetadata.getName) + .withUid(driverPod.getMetadata.getUid) + .endOwnerReference() + .endMetadata() + .withNewSpec() + .withHostname(hostname) + .withRestartPolicy("Never") + .withNodeSelector(nodeSelector.asJava) + .endSpec() + .build() + + val containerWithExecutorLimitCores = executorLimitCores.map { + limitCores => + val executorCpuLimitQuantity = new QuantityBuilder(false) + .withAmount(limitCores) + .build() + new ContainerBuilder(executorContainer) + .editResources() + .addToLimits("cpu", executorCpuLimitQuantity) + .endResources() + .build() + }.getOrElse(executorContainer) + + new PodBuilder(executorPod) + .editSpec() + .addToContainers(containerWithExecutorLimitCores) + .endSpec() + .build() + } +} + +private object ExecutorPodFactoryImpl { + private val DEFAULT_STATIC_PORT = 10000 +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala new file mode 100644 index 0000000000000..034180125efb4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -0,0 +1,69 @@ +/* + * 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.scheduler.cluster.k8s + +import java.io.File + +import io.fabric8.kubernetes.client.Config + +import org.apache.spark.SparkContext +import org.apache.spark.deploy.k8s.{ConfigurationUtils, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.internal.Logging +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} +import org.apache.spark.util.{ThreadUtils, Utils} + +private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { + + override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") + + override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + new TaskSchedulerImpl(sc) + } + + override def createSchedulerBackend(sc: SparkContext, masterURL: String, scheduler: TaskScheduler) + : SchedulerBackend = { + val sparkConf = sc.getConf + + val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( + KUBERNETES_MASTER_INTERNAL_URL, + Some(sparkConf.get(KUBERNETES_NAMESPACE)), + APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX, + sparkConf, + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) + + val executorPodFactory = new ExecutorPodFactoryImpl(sparkConf) + val allocatorExecutor = ThreadUtils + .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") + val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( + "kubernetes-executor-requests") + new KubernetesClusterSchedulerBackend( + scheduler.asInstanceOf[TaskSchedulerImpl], + sc.env.rpcEnv, + executorPodFactory, + kubernetesClient, + allocatorExecutor, + requestExecutorsService) + } + + override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala new file mode 100644 index 0000000000000..8a53606c89fd3 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -0,0 +1,456 @@ +/* + * 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.scheduler.cluster.k8s + +import java.io.Closeable +import java.net.InetAddress +import java.util.concurrent.{ConcurrentHashMap, ExecutorService, ScheduledExecutorService, TimeUnit} +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} + +import scala.collection.{concurrent, mutable} +import scala.collection.JavaConverters._ +import scala.concurrent.{ExecutionContext, Future} + +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.SparkException +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv} +import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.Utils + +private[spark] class KubernetesClusterSchedulerBackend( + scheduler: TaskSchedulerImpl, + rpcEnv: RpcEnv, + executorPodFactory: ExecutorPodFactory, + kubernetesClient: KubernetesClient, + allocatorExecutor: ScheduledExecutorService, + requestExecutorsService: ExecutorService) + extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { + + import KubernetesClusterSchedulerBackend._ + + private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) + private val RUNNING_EXECUTOR_PODS_LOCK = new Object + // Indexed by executor IDs and guarded by RUNNING_EXECUTOR_PODS_LOCK. + private val runningExecutorsToPods = new mutable.HashMap[String, Pod] + // Indexed by executor pod names and guarded by RUNNING_EXECUTOR_PODS_LOCK. + private val runningPodsToExecutors = new mutable.HashMap[String, String] + private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]() + private val podsWithKnownExitReasons = new ConcurrentHashMap[String, ExecutorExited]() + private val disconnectedPodsByExecutorIdPendingRemoval = new ConcurrentHashMap[String, Pod]() + + private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) + + private val kubernetesDriverPodName = conf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse( + throw new SparkException("Must specify the driver pod name")) + private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( + requestExecutorsService) + + private val driverPod = try { + kubernetesClient.pods() + .inNamespace(kubernetesNamespace) + .withName(kubernetesDriverPodName) + .get() + } catch { + case throwable: Throwable => + logError(s"Executor cannot find driver pod.", throwable) + throw new SparkException(s"Executor cannot find driver pod", throwable) + } + + override val minRegisteredRatio = + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + 0.8 + } else { + super.minRegisteredRatio + } + + private val executorWatchResource = new AtomicReference[Closeable] + protected var totalExpectedExecutors = new AtomicInteger(0) + + private val driverUrl = RpcEndpointAddress( + conf.get("spark.driver.host"), + conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + + private val initialExecutors = getInitialTargetExecutorNumber() + + private val podAllocationInterval = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) + require(podAllocationInterval > 0, s"Allocation batch delay " + + s"${KUBERNETES_ALLOCATION_BATCH_DELAY} " + + s"is ${podAllocationInterval}, should be a positive integer") + + private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) + require(podAllocationSize > 0, s"Allocation batch size " + + s"${KUBERNETES_ALLOCATION_BATCH_SIZE} " + + s"is ${podAllocationSize}, should be a positive integer") + + private val allocatorRunnable = new Runnable { + + // Maintains a map of executor id to count of checks performed to learn the loss reason + // for an executor. + private val executorReasonCheckAttemptCounts = new mutable.HashMap[String, Int] + + override def run(): Unit = { + handleDisconnectedExecutors() + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + if (totalRegisteredExecutors.get() < runningExecutorsToPods.size) { + logDebug("Waiting for pending executors before scaling") + } else if (totalExpectedExecutors.get() <= runningExecutorsToPods.size) { + logDebug("Maximum allowed executor limit reached. Not scaling up further.") + } else { + val nodeToLocalTaskCount = getNodesWithLocalTaskCounts + for (i <- 0 until math.min( + totalExpectedExecutors.get - runningExecutorsToPods.size, podAllocationSize)) { + val (executorId, pod) = allocateNewExecutorPod(nodeToLocalTaskCount) + runningExecutorsToPods.put(executorId, pod) + runningPodsToExecutors.put(pod.getMetadata.getName, executorId) + logInfo( + s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}") + } + } + } + } + + def handleDisconnectedExecutors(): Unit = { + // For each disconnected executor, synchronize with the loss reasons that may have been found + // by the executor pod watcher. If the loss reason was discovered by the watcher, + // inform the parent class with removeExecutor. + disconnectedPodsByExecutorIdPendingRemoval.keys().asScala.foreach { case (executorId) => + val executorPod = disconnectedPodsByExecutorIdPendingRemoval.get(executorId) + val knownExitReason = Option(podsWithKnownExitReasons.remove( + executorPod.getMetadata.getName)) + knownExitReason.fold { + removeExecutorOrIncrementLossReasonCheckCount(executorId) + } { executorExited => + logDebug(s"Removing executor $executorId with loss reason " + executorExited.message) + removeExecutor(executorId, executorExited) + // We keep around executors that have exit conditions caused by the application. This + // allows them to be debugged later on. Otherwise, mark them as to be deleted from the + // the API server. + if (!executorExited.exitCausedByApp) { + deleteExecutorFromClusterAndDataStructures(executorId) + } + } + } + } + + def removeExecutorOrIncrementLossReasonCheckCount(executorId: String): Unit = { + val reasonCheckCount = executorReasonCheckAttemptCounts.getOrElse(executorId, 0) + if (reasonCheckCount >= MAX_EXECUTOR_LOST_REASON_CHECKS) { + removeExecutor(executorId, SlaveLost("Executor lost for unknown reasons.")) + deleteExecutorFromClusterAndDataStructures(executorId) + } else { + executorReasonCheckAttemptCounts.put(executorId, reasonCheckCount + 1) + } + } + + def deleteExecutorFromClusterAndDataStructures(executorId: String): Unit = { + disconnectedPodsByExecutorIdPendingRemoval.remove(executorId) + executorReasonCheckAttemptCounts -= executorId + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + runningExecutorsToPods.remove(executorId).map { pod => + kubernetesClient.pods().delete(pod) + runningPodsToExecutors.remove(pod.getMetadata.getName) + }.getOrElse(logWarning(s"Unable to remove pod for unknown executor $executorId")) + } + } + } + + private def getInitialTargetExecutorNumber(defaultNumExecutors: Int = 1): Int = { + if (Utils.isDynamicAllocationEnabled(conf)) { + val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) + val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) + val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", 1) + require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, + s"initial executor number $initialNumExecutors must between min executor number " + + s"$minNumExecutors and max executor number $maxNumExecutors") + + initialNumExecutors + } else { + conf.getInt("spark.executor.instances", defaultNumExecutors) + } + + } + + override def applicationId(): String = conf.get("spark.app.id", super.applicationId()) + + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio + } + + override def start(): Unit = { + super.start() + executorWatchResource.set( + kubernetesClient + .pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId()) + .watch(new ExecutorPodsWatcher())) + + allocatorExecutor.scheduleWithFixedDelay( + allocatorRunnable, 0L, podAllocationInterval, TimeUnit.SECONDS) + + if (!Utils.isDynamicAllocationEnabled(conf)) { + doRequestTotalExecutors(initialExecutors) + } + } + + override def stop(): Unit = { + // stop allocation of new resources and caches. + allocatorExecutor.shutdown() + + // send stop message to executors so they shut down cleanly + super.stop() + + // then delete the executor pods + // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context. + // When using Utils.tryLogNonFatalError some of the code fails but without any logs or + // indication as to why. + try { + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_)) + runningExecutorsToPods.clear() + runningPodsToExecutors.clear() + } + executorPodsByIPs.clear() + val resource = executorWatchResource.getAndSet(null) + if (resource != null) { + resource.close() + } + } catch { + case e: Throwable => logError("Uncaught exception while shutting down controllers.", e) + } + try { + logInfo("Closing kubernetes client") + kubernetesClient.close() + } catch { + case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e) + } + } + + /** + * @return A map of K8s cluster nodes to the number of tasks that could benefit from data + * locality if an executor launches on the cluster node. + */ + private def getNodesWithLocalTaskCounts() : Map[String, Int] = { + val nodeToLocalTaskCount = mutable.Map[String, Int]() ++ + KubernetesClusterSchedulerBackend.this.synchronized { + hostToLocalTaskCount + } + for (pod <- executorPodsByIPs.values().asScala) { + // Remove cluster nodes that are running our executors already. + // TODO: This prefers spreading out executors across nodes. In case users want + // consolidating executors on fewer nodes, introduce a flag. See the spark.deploy.spreadOut + // flag that Spark standalone has: https://spark.apache.org/docs/latest/spark-standalone.html + nodeToLocalTaskCount.remove(pod.getSpec.getNodeName).nonEmpty || + nodeToLocalTaskCount.remove(pod.getStatus.getHostIP).nonEmpty || + nodeToLocalTaskCount.remove( + InetAddress.getByName(pod.getStatus.getHostIP).getCanonicalHostName).nonEmpty + } + nodeToLocalTaskCount.toMap[String, Int] + } + + /** + * Allocates a new executor pod + * + * @param nodeToLocalTaskCount A map of K8s cluster nodes to the number of tasks that could + * benefit from data locality if an executor launches on the cluster + * node. + * @return A tuple of the new executor name and the Pod data structure. + */ + private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = { + val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString + val executorPod = executorPodFactory.createExecutorPod( + executorId, + applicationId(), + driverUrl, + conf.getExecutorEnv, + driverPod, + nodeToLocalTaskCount) + try { + (executorId, kubernetesClient.pods.create(executorPod)) + } catch { + case throwable: Throwable => + logError("Failed to allocate executor pod.", throwable) + throw throwable + } + } + + override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { + totalExpectedExecutors.set(requestedTotal) + true + } + + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + for (executor <- executorIds) { + val maybeRemovedExecutor = runningExecutorsToPods.remove(executor) + maybeRemovedExecutor.foreach { executorPod => + kubernetesClient.pods().delete(executorPod) + disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod) + runningPodsToExecutors.remove(executorPod.getMetadata.getName) + } + if (maybeRemovedExecutor.isEmpty) { + logWarning(s"Unable to remove pod for unknown executor $executor") + } + } + } + true + } + + def getExecutorPodByIP(podIP: String): Option[Pod] = { + // Note: Per https://github.com/databricks/scala-style-guide#concurrency, we don't + // want to be switching to scala.collection.concurrent.Map on + // executorPodsByIPs. + val pod = executorPodsByIPs.get(podIP) + Option(pod) + } + + private class ExecutorPodsWatcher extends Watcher[Pod] { + + private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1 + + override def eventReceived(action: Action, pod: Pod): Unit = { + if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running" + && pod.getMetadata.getDeletionTimestamp == null) { + val podIP = pod.getStatus.getPodIP + val clusterNodeName = pod.getSpec.getNodeName + logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") + executorPodsByIPs.put(podIP, pod) + } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) || + action == Action.DELETED || action == Action.ERROR) { + val podName = pod.getMetadata.getName + val podIP = pod.getStatus.getPodIP + logDebug(s"Executor pod $podName at IP $podIP was at $action.") + if (podIP != null) { + executorPodsByIPs.remove(podIP) + } + if (action == Action.ERROR) { + logInfo(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) + handleErroredPod(pod) + } else if (action == Action.DELETED) { + logInfo(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason) + handleDeletedPod(pod) + } + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logDebug("Executor pod watch closed.", cause) + } + + def getExecutorExitStatus(pod: Pod): Int = { + val containerStatuses = pod.getStatus.getContainerStatuses + if (!containerStatuses.isEmpty) { + // we assume the first container represents the pod status. This assumption may not hold + // true in the future. Revisit this if side-car containers start running inside executor + // pods. + getExecutorExitStatus(containerStatuses.get(0)) + } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS + } + + def getExecutorExitStatus(containerStatus: ContainerStatus): Int = { + Option(containerStatus.getState).map(containerState => + Option(containerState.getTerminated).map(containerStateTerminated => + containerStateTerminated.getExitCode.intValue()).getOrElse(UNKNOWN_EXIT_CODE) + ).getOrElse(UNKNOWN_EXIT_CODE) + } + + def isPodAlreadyReleased(pod: Pod): Boolean = { + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + !runningPodsToExecutors.contains(pod.getMetadata.getName) + } + } + + def handleErroredPod(pod: Pod): Unit = { + val containerExitStatus = getExecutorExitStatus(pod) + // container was probably actively killed by the driver. + val exitReason = if (isPodAlreadyReleased(pod)) { + ExecutorExited(containerExitStatus, exitCausedByApp = false, + s"Container in pod " + pod.getMetadata.getName + + " exited from explicit termination request.") + } else { + val containerExitReason = containerExitStatus match { + case VMEM_EXCEEDED_EXIT_CODE | PMEM_EXCEEDED_EXIT_CODE => + memLimitExceededLogMessage(pod.getStatus.getReason) + case _ => + // Here we can't be sure that that exit was caused by the application but this seems + // to be the right default since we know the pod was not explicitly deleted by + // the user. + s"Pod ${pod.getMetadata.getName}'s executor container exited with exit status" + + s" code $containerExitStatus." + } + ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason) + } + podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason) + } + + def handleDeletedPod(pod: Pod): Unit = { + val exitMessage = if (isPodAlreadyReleased(pod)) { + s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request." + } else { + s"Pod ${pod.getMetadata.getName} deleted or lost." + } + val exitReason = ExecutorExited( + getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage) + podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason) + } + } + + override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { + new KubernetesDriverEndpoint(rpcEnv, properties) + } + + private class KubernetesDriverEndpoint( + rpcEnv: RpcEnv, + sparkProperties: Seq[(String, String)]) + extends DriverEndpoint(rpcEnv, sparkProperties) { + + override def onDisconnected(rpcAddress: RpcAddress): Unit = { + addressToExecutorId.get(rpcAddress).foreach { executorId => + if (disableExecutor(executorId)) { + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + runningExecutorsToPods.get(executorId).foreach { pod => + disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) + } + } + } + } + } + } +} + +private object KubernetesClusterSchedulerBackend { + private val VMEM_EXCEEDED_EXIT_CODE = -103 + private val PMEM_EXCEEDED_EXIT_CODE = -104 + private val UNKNOWN_EXIT_CODE = -111 + // Number of times we are allowed check for the loss reason for an executor before we give up + // and assume the executor failed for good, and attribute it to a framework fault. + val MAX_EXECUTOR_LOST_REASON_CHECKS = 10 + + def memLimitExceededLogMessage(diagnostics: String): String = { + s"Pod/Container killed for exceeding memory limits. $diagnostics" + + " Consider boosting spark executor memory overhead." + } +} + diff --git a/resource-managers/kubernetes/core/src/test/resources/log4j.properties b/resource-managers/kubernetes/core/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..ad95fadb7c0c0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/resources/log4j.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from a few verbose libraries. +log4j.logger.com.sun.jersey=WARN +log4j.logger.org.apache.hadoop=WARN +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.mortbay=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala new file mode 100644 index 0000000000000..b3954f88f1457 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -0,0 +1,145 @@ +/* + * 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.scheduler.cluster.k8s + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{Pod, VolumeBuilder, VolumeMountBuilder, _} +import io.fabric8.kubernetes.client.KubernetesClient +import org.apache.commons.io.FilenameUtils +import org.mockito.{AdditionalAnswers, MockitoAnnotations} +import org.mockito.Matchers.{any, eq => mockitoEq} +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants +import org.apache.spark.scheduler.cluster.k8s.ExecutorPodFactoryImpl + +class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { + private val driverPodName: String = "driver-pod" + private val driverPodUid: String = "driver-uid" + private val driverUrl: String = "driver-url" + private val executorPrefix: String = "base" + private val executorImage: String = "executor-image" + private val driverPod = new PodBuilder() + .withNewMetadata() + .withName(driverPodName) + .withUid(driverPodUid) + .endMetadata() + .withNewSpec() + .withNodeName("some-node") + .endSpec() + .withNewStatus() + .withHostIP("192.168.99.100") + .endStatus() + .build() + private var baseConf: SparkConf = _ + + before { + MockitoAnnotations.initMocks(this) + baseConf = new SparkConf() + .set(KUBERNETES_DRIVER_POD_NAME, driverPodName) + .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, executorPrefix) + .set(EXECUTOR_DOCKER_IMAGE, executorImage) + } + private var kubernetesClient: KubernetesClient = _ + + test("basic executor pod has reasonable defaults") { + val factory = new ExecutorPodFactoryImpl(baseConf) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + // The executor pod name and default labels. + assert(executor.getMetadata.getName === s"$executorPrefix-exec-1") + assert(executor.getMetadata.getLabels.size() === 3) + + // There is exactly 1 container with no volume mounts and default memory limits. + // Default memory limit is 1024M + 384M (minimum overhead constant). + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getImage === executorImage) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.isEmpty) + assert(executor.getSpec.getContainers.get(0).getResources.getLimits.size() === 1) + assert(executor.getSpec.getContainers.get(0).getResources + .getLimits.get("memory").getAmount === "1408Mi") + + // The pod has no node selector, volumes. + assert(executor.getSpec.getNodeSelector.isEmpty) + assert(executor.getSpec.getVolumes.isEmpty) + + checkEnv(executor, Map()) + checkOwnerReferences(executor, driverPodUid) + } + + test("executor pod hostnames get truncated to 63 characters") { + val conf = baseConf.clone() + conf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, + "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple") + + val factory = new ExecutorPodFactoryImpl(conf) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + assert(executor.getSpec.getHostname.length === 63) + } + + test("classpath and extra java options get translated into environment variables") { + val conf = baseConf.clone() + conf.set(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS, "foo=bar") + conf.set(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH, "bar=baz") + + val factory = new ExecutorPodFactoryImpl(conf) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) + + checkEnv(executor, + Map("SPARK_JAVA_OPT_0" -> "foo=bar", + "SPARK_EXECUTOR_EXTRA_CLASSPATH" -> "bar=baz", + "qux" -> "quux")) + checkOwnerReferences(executor, driverPodUid) + } + + // There is always exactly one controller reference, and it points to the driver pod. + private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { + assert(executor.getMetadata.getOwnerReferences.size() === 1) + assert(executor.getMetadata.getOwnerReferences.get(0).getUid === driverPodUid) + assert(executor.getMetadata.getOwnerReferences.get(0).getController === true) + } + + // Check that the expected environment variables are present. + private def checkEnv(executor: Pod, additionalEnvVars: Map[String, String]): Unit = { + val defaultEnvs = Map( + constants.ENV_EXECUTOR_ID -> "1", + constants.ENV_DRIVER_URL -> "dummy", + constants.ENV_EXECUTOR_CORES -> "1", + constants.ENV_EXECUTOR_MEMORY -> "1g", + constants.ENV_APPLICATION_ID -> "dummy", + constants.ENV_MOUNTED_CLASSPATH -> "/var/spark-data/spark-jars/*", + constants.ENV_EXECUTOR_POD_IP -> null, + constants.ENV_EXECUTOR_PORT -> "10000") ++ additionalEnvVars + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getEnv().size() === defaultEnvs.size) + val mapEnvs = executor.getSpec.getContainers.get(0).getEnv.asScala.map { + x => (x.getName, x.getValue) + }.toMap + assert(defaultEnvs === mapEnvs) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala new file mode 100644 index 0000000000000..6328a781056cb --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -0,0 +1,378 @@ +/* + * 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.scheduler.cluster.k8s + +import java.util.concurrent.{ExecutorService, ScheduledExecutorService, TimeUnit} + +import scala.collection.JavaConverters._ +import scala.concurrent.Future + +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList} +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource} +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Matchers.{any, eq => mockitoEq} +import org.mockito.Mockito.{mock => _, _} +import org.scalatest.BeforeAndAfter +import org.scalatest.mock.MockitoSugar._ + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.rpc._ +import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend + +private[spark] class KubernetesClusterSchedulerBackendSuite + extends SparkFunSuite with BeforeAndAfter { + + private val APP_ID = "test-spark-app" + private val DRIVER_POD_NAME = "spark-driver-pod" + private val NAMESPACE = "test-namespace" + private val SPARK_DRIVER_HOST = "localhost" + private val SPARK_DRIVER_PORT = 7077 + private val POD_ALLOCATION_INTERVAL = 60L + private val DRIVER_URL = RpcEndpointAddress( + SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + private val FIRST_EXECUTOR_POD = new PodBuilder() + .withNewMetadata() + .withName("pod1") + .endMetadata() + .withNewSpec() + .withNodeName("node1") + .endSpec() + .withNewStatus() + .withHostIP("192.168.99.100") + .endStatus() + .build() + private val SECOND_EXECUTOR_POD = new PodBuilder() + .withNewMetadata() + .withName("pod2") + .endMetadata() + .withNewSpec() + .withNodeName("node2") + .endSpec() + .withNewStatus() + .withHostIP("192.168.99.101") + .endStatus() + .build() + + private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + private type LABELLED_PODS = FilterWatchListDeletable[ + Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]] + private type IN_NAMESPACE_PODS = NonNamespaceOperation[ + Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + + @Mock + private var sparkContext: SparkContext = _ + + @Mock + private var listenerBus: LiveListenerBus = _ + + @Mock + private var taskSchedulerImpl: TaskSchedulerImpl = _ + + @Mock + private var allocatorExecutor: ScheduledExecutorService = _ + + @Mock + private var requestExecutorsService: ExecutorService = _ + + @Mock + private var executorPodFactory: ExecutorPodFactory = _ + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: PODS = _ + + @Mock + private var podsWithLabelOperations: LABELLED_PODS = _ + + @Mock + private var podsInNamespace: IN_NAMESPACE_PODS = _ + + @Mock + private var podsWithDriverName: PodResource[Pod, DoneablePod] = _ + + @Mock + private var rpcEnv: RpcEnv = _ + + @Mock + private var driverEndpointRef: RpcEndpointRef = _ + + @Mock + private var executorPodsWatch: Watch = _ + + private var sparkConf: SparkConf = _ + private var executorPodsWatcherArgument: ArgumentCaptor[Watcher[Pod]] = _ + private var allocatorRunnable: ArgumentCaptor[Runnable] = _ + private var requestExecutorRunnable: ArgumentCaptor[Runnable] = _ + private var driverEndpoint: ArgumentCaptor[RpcEndpoint] = _ + + private val driverPod = new PodBuilder() + .withNewMetadata() + .withName(DRIVER_POD_NAME) + .addToLabels(SPARK_APP_ID_LABEL, APP_ID) + .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_DRIVER_ROLE) + .endMetadata() + .build() + + before { + MockitoAnnotations.initMocks(this) + sparkConf = new SparkConf() + .set("spark.app.id", APP_ID) + .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) + .set(KUBERNETES_NAMESPACE, NAMESPACE) + .set("spark.driver.host", SPARK_DRIVER_HOST) + .set("spark.driver.port", SPARK_DRIVER_PORT.toString) + .set(KUBERNETES_ALLOCATION_BATCH_DELAY, POD_ALLOCATION_INTERVAL) + executorPodsWatcherArgument = ArgumentCaptor.forClass(classOf[Watcher[Pod]]) + allocatorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) + requestExecutorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) + driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) + when(sparkContext.conf).thenReturn(sparkConf) + when(sparkContext.listenerBus).thenReturn(listenerBus) + when(taskSchedulerImpl.sc).thenReturn(sparkContext) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withLabel(SPARK_APP_ID_LABEL, APP_ID)).thenReturn(podsWithLabelOperations) + when(podsWithLabelOperations.watch(executorPodsWatcherArgument.capture())) + .thenReturn(executorPodsWatch) + when(podOperations.inNamespace(NAMESPACE)).thenReturn(podsInNamespace) + when(podsInNamespace.withName(DRIVER_POD_NAME)).thenReturn(podsWithDriverName) + when(podsWithDriverName.get()).thenReturn(driverPod) + when(allocatorExecutor.scheduleWithFixedDelay( + allocatorRunnable.capture(), + mockitoEq(0L), + mockitoEq(POD_ALLOCATION_INTERVAL), + mockitoEq(TimeUnit.SECONDS))).thenReturn(null) + // Creating Futures in Scala backed by a Java executor service resolves to running + // ExecutorService#execute (as opposed to submit) + doNothing().when(requestExecutorsService).execute(requestExecutorRunnable.capture()) + when(rpcEnv.setupEndpoint( + mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) + .thenReturn(driverEndpointRef) + when(driverEndpointRef.ask[Boolean] + (any(classOf[Any])) + (any())).thenReturn(mock[Future[Boolean]]) + } + + test("Basic lifecycle expectations when starting and stopping the scheduler.") { + val scheduler = newSchedulerBackend() + scheduler.start() + assert(executorPodsWatcherArgument.getValue != null) + assert(allocatorRunnable.getValue != null) + scheduler.stop() + verify(executorPodsWatch).close() + } + + test("Static allocation should request executors upon first allocator run.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + val scheduler = newSchedulerBackend() + scheduler.start() + requestExecutorRunnable.getValue.run() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + allocatorRunnable.getValue.run() + verify(podOperations).create(FIRST_EXECUTOR_POD) + verify(podOperations).create(SECOND_EXECUTOR_POD) + } + + test("Killing executors deletes the executor pods") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + val scheduler = newSchedulerBackend() + scheduler.start() + requestExecutorRunnable.getValue.run() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + allocatorRunnable.getValue.run() + scheduler.doKillExecutors(Seq("2")) + requestExecutorRunnable.getAllValues.asScala.last.run() + verify(podOperations).delete(SECOND_EXECUTOR_POD) + verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + } + + test("Executors should be requested in batches.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + val scheduler = newSchedulerBackend() + scheduler.start() + requestExecutorRunnable.getValue.run() + when(podOperations.create(any(classOf[Pod]))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + allocatorRunnable.getValue.run() + verify(podOperations).create(FIRST_EXECUTOR_POD) + verify(podOperations, never()).create(SECOND_EXECUTOR_POD) + val registerFirstExecutorMessage = RegisterExecutor( + "1", mock[RpcEndpointRef], "localhost", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + allocatorRunnable.getValue.run() + verify(podOperations).create(SECOND_EXECUTOR_POD) + } + + test("Deleting executors and then running an allocator pass after finding the loss reason" + + " should only delete the pod once.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend() + scheduler.start() + requestExecutorRunnable.getValue.run() + when(podOperations.create(any(classOf[Pod]))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + allocatorRunnable.getValue.run() + val executorEndpointRef = mock[RpcEndpointRef] + when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) + val registerFirstExecutorMessage = RegisterExecutor( + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + scheduler.doRequestTotalExecutors(0) + requestExecutorRunnable.getAllValues.asScala.last.run() + scheduler.doKillExecutors(Seq("1")) + requestExecutorRunnable.getAllValues.asScala.last.run() + verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) + + val exitedPod = exitPod(FIRST_EXECUTOR_POD, 0) + executorPodsWatcherArgument.getValue.eventReceived(Action.DELETED, exitedPod) + allocatorRunnable.getValue.run() + verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) + verify(driverEndpointRef, times(1)).ask[Boolean]( + RemoveExecutor("1", ExecutorExited( + 0, + exitCausedByApp = false, + s"Container in pod ${exitedPod.getMetadata.getName} exited from" + + s" explicit termination request."))) + } + + test("Executors that disconnect from application errors are noted as exits caused by app.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend() + scheduler.start() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getValue.run() + val executorEndpointRef = mock[RpcEndpointRef] + when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) + val registerFirstExecutorMessage = RegisterExecutor( + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) + executorPodsWatcherArgument.getValue.eventReceived( + Action.ERROR, exitPod(FIRST_EXECUTOR_POD, 1)) + + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + scheduler.doRequestTotalExecutors(1) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getAllValues.asScala.last.run() + verify(driverEndpointRef).ask[Boolean]( + RemoveExecutor("1", ExecutorExited( + 1, + exitCausedByApp = true, + s"Pod ${FIRST_EXECUTOR_POD.getMetadata.getName}'s executor container exited with" + + " exit status code 1."))) + verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + } + + test("Executors should only try to get the loss reason a number of times before giving up and" + + " removing the executor.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend() + scheduler.start() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getValue.run() + val executorEndpointRef = mock[RpcEndpointRef] + when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) + val registerFirstExecutorMessage = RegisterExecutor( + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) + 1 to KubernetesClusterSchedulerBackend.MAX_EXECUTOR_LOST_REASON_CHECKS foreach { _ => + allocatorRunnable.getValue.run() + verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + } + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + allocatorRunnable.getValue.run() + verify(podOperations).delete(FIRST_EXECUTOR_POD) + verify(driverEndpointRef).ask[Boolean]( + RemoveExecutor("1", SlaveLost("Executor lost for unknown reasons."))) + } + + private def newSchedulerBackend(): KubernetesClusterSchedulerBackend = { + new KubernetesClusterSchedulerBackend( + taskSchedulerImpl, + rpcEnv, + executorPodFactory, + kubernetesClient, + allocatorExecutor, + requestExecutorsService) + } + + private def exitPod(basePod: Pod, exitCode: Int): Pod = { + new PodBuilder(FIRST_EXECUTOR_POD) + .editStatus() + .addNewContainerStatus() + .withNewState() + .withNewTerminated() + .withExitCode(exitCode) + .endTerminated() + .endState() + .endContainerStatus() + .endStatus() + .build() + } + + private def expectPodCreationWithId(executorId: Int, expectedPod: Pod): Unit = { + when(executorPodFactory.createExecutorPod( + executorId.toString, + APP_ID, + DRIVER_URL, + sparkConf.getExecutorEnv, + driverPod, + Map.empty)).thenReturn(expectedPod) + } + +} From 75e31a9f6e52dc4784b3efad1a45c42181cabe4a Mon Sep 17 00:00:00 2001 From: foxish Date: Tue, 17 Oct 2017 12:14:38 -0700 Subject: [PATCH 02/29] Adding to modules.py and SparkBuild.scala --- dev/sparktestsupport/modules.py | 8 ++++++++ project/SparkBuild.scala | 4 ++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 91d5667ed1f07..46a16f83073f7 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -531,6 +531,14 @@ def __hash__(self): sbt_test_goals=["mesos/test"] ) +kubernetes = Module( + name="kubernetes", + dependencies=[], + source_file_regexes=["resource-managers/kubernetes/core"], + build_profile_flags=["-Pkubernetes"], + sbt_test_goals=["kubernetes/test"] +) + # The root module is a dummy module which is used to run all of the tests. # No other modules should directly depend on this module. root = Module( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9501eed1e906b..d9b17712e6f22 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -53,11 +53,11 @@ object BuildCommons { "tags", "sketch", "kvstore" ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects - val optionallyEnabledProjects@Seq(mesos, yarn, + val optionallyEnabledProjects@Seq(kubernetes, mesos, yarn, streamingFlumeSink, streamingFlume, streamingKafka, sparkGangliaLgpl, streamingKinesisAsl, dockerIntegrationTests, hadoopCloud) = - Seq("mesos", "yarn", + Seq("kubernetes", "mesos", "yarn", "streaming-flume-sink", "streaming-flume", "streaming-kafka-0-8", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests", "hadoop-cloud").map(ProjectRef(buildLocation, _)) From cf82b21e04c509d08e56faba1d0640e4b9571aaa Mon Sep 17 00:00:00 2001 From: foxish Date: Tue, 17 Oct 2017 13:39:17 -0700 Subject: [PATCH 03/29] Exclude from unidoc, update travis --- .travis.yml | 2 +- project/SparkBuild.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index d7e9f8c0290e8..05b94adeeb93b 100644 --- a/.travis.yml +++ b/.travis.yml @@ -43,7 +43,7 @@ notifications: # 5. Run maven install before running lint-java. install: - export MAVEN_SKIP_RC=1 - - build/mvn -T 4 -q -DskipTests -Pmesos -Pyarn -Pkinesis-asl -Phive -Phive-thriftserver install + - build/mvn -T 4 -q -DskipTests -Pkubernetes -Pmesos -Pyarn -Pkinesis-asl -Phive -Phive-thriftserver install # 6. Run lint-java. script: diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d9b17712e6f22..14e6c586b18f6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -671,9 +671,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags, streamingKafka010, sqlKafka010), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, kubernetes, yarn, tags, streamingKafka010, sqlKafka010), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags, streamingKafka010, sqlKafka010), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, kubernetes, yarn, tags, streamingKafka010, sqlKafka010), unidocAllClasspaths in (ScalaUnidoc, unidoc) := { ignoreClasspaths((unidocAllClasspaths in (ScalaUnidoc, unidoc)).value) From 488c535a16e9003aa9b73d209ede0a19e5658146 Mon Sep 17 00:00:00 2001 From: foxish Date: Tue, 17 Oct 2017 15:42:16 -0700 Subject: [PATCH 04/29] Address a bunch of style and other comments --- .../spark/deploy/k8s/ConfigurationUtils.scala | 25 ++++++++++-- .../spark/deploy/k8s/OptionRequirements.scala | 40 ------------------- .../k8s/SparkKubernetesClientFactory.scala | 2 +- .../org/apache/spark/deploy/k8s/config.scala | 20 ---------- .../apache/spark/deploy/k8s/constants.scala | 28 ------------- .../cluster/k8s/ExecutorPodFactory.scala | 5 +-- .../k8s/KubernetesClusterManager.scala | 5 +-- .../KubernetesClusterSchedulerBackend.scala | 2 +- .../cluster/k8s/ExecutorPodFactorySuite.scala | 10 +---- ...bernetesClusterSchedulerBackendSuite.scala | 6 +-- 10 files changed, 33 insertions(+), 110 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index eb0e4328acae0..23832e0850c10 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -17,10 +17,9 @@ package org.apache.spark.deploy.k8s -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.internal.Logging +import org.apache.spark.SparkConf -private[spark] object ConfigurationUtils extends Logging { +private[spark] object ConfigurationUtils { def parsePrefixedKeyValuePairs( sparkConf: SparkConf, prefix: String, @@ -34,4 +33,24 @@ private[spark] object ConfigurationUtils extends Logging { } fromPrefix.toMap } + + def requireBothOrNeitherDefined( + opt1: Option[_], + opt2: Option[_], + errMessageWhenFirstIsMissing: String, + errMessageWhenSecondIsMissing: String): Unit = { + requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) + requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) + } + + def requireSecondIfFirstIsDefined( + opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { + opt1.foreach { _ => + require(opt2.isDefined, errMessageWhenSecondIsMissing) + } + } + + def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { + opt1.foreach { _ => require(opt2.isEmpty, errMessage) } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala deleted file mode 100644 index 89053de5b9558..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.deploy.k8s - -private[spark] object OptionRequirements { - - def requireBothOrNeitherDefined( - opt1: Option[_], - opt2: Option[_], - errMessageWhenFirstIsMissing: String, - errMessageWhenSecondIsMissing: String): Unit = { - requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) - requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) - } - - def requireSecondIfFirstIsDefined( - opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { - opt1.foreach { _ => - require(opt2.isDefined, errMessageWhenSecondIsMissing) - } - } - - def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { - opt1.foreach { _ => require(opt2.isEmpty, errMessage) } - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index c90303b6ba506..864c26959f929 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -48,7 +48,7 @@ private[spark] object SparkKubernetesClientFactory { .map(new File(_)) .orElse(maybeServiceAccountToken) val oauthTokenValue = sparkConf.getOption(oauthTokenConf) - OptionRequirements.requireNandDefined( + ConfigurationUtils.requireNandDefined( oauthTokenFile, oauthTokenValue, s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala index e05b9d99c962f..5c66a125bf114 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala @@ -31,12 +31,6 @@ package object config extends Logging { .stringConf .createWithDefault("default") - private[spark] val DRIVER_DOCKER_IMAGE = - ConfigBuilder("spark.kubernetes.driver.docker.image") - .doc("Docker image to use for the driver. Specify this using the standard Docker tag format.") - .stringConf - .createWithDefault(s"spark-driver:$sparkVersion") - private[spark] val EXECUTOR_DOCKER_IMAGE = ConfigBuilder("spark.kubernetes.executor.docker.image") .doc("Docker image to use for the executors. Specify this using the standard Docker tag" + @@ -81,22 +75,8 @@ package object config extends Logging { .bytesConf(ByteUnit.MiB) .createOptional - private[spark] val KUBERNETES_DRIVER_MEMORY_OVERHEAD = - ConfigBuilder("spark.kubernetes.driver.memoryOverhead") - .doc("The amount of off-heap memory (in megabytes) to be allocated for the driver and the" + - " driver submission server. This is memory that accounts for things like VM overheads," + - " interned strings, other native overheads, etc. This tends to grow with the driver's" + - " memory size (typically 6-10%).") - .bytesConf(ByteUnit.MiB) - .createOptional - - private[spark] val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label." - private[spark] val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation." private[spark] val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." private[spark] val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." - private[spark] val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv." - private[spark] val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets." - private[spark] val KUBERNETES_EXECUTOR_SECRETS_PREFIX = "spark.kubernetes.executor.secrets." private[spark] val KUBERNETES_DRIVER_POD_NAME = ConfigBuilder("spark.kubernetes.driver.pod.name") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala index 611afe4883854..664e645019820 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala @@ -24,29 +24,10 @@ package object constants { private[spark] val SPARK_POD_DRIVER_ROLE = "driver" private[spark] val SPARK_POD_EXECUTOR_ROLE = "executor" - // Credentials secrets - private[spark] val DRIVER_CREDENTIALS_SECRETS_BASE_DIR = - "/mnt/secrets/spark-kubernetes-credentials" - private[spark] val DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME = "ca-cert" - private[spark] val DRIVER_CREDENTIALS_CA_CERT_PATH = - s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME" - private[spark] val DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME = "client-key" - private[spark] val DRIVER_CREDENTIALS_CLIENT_KEY_PATH = - s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME" - private[spark] val DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME = "client-cert" - private[spark] val DRIVER_CREDENTIALS_CLIENT_CERT_PATH = - s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME" - private[spark] val DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME = "oauth-token" - private[spark] val DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH = - s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME" - private[spark] val DRIVER_CREDENTIALS_SECRET_VOLUME_NAME = "kubernetes-credentials" - // Default and fixed ports private[spark] val DEFAULT_DRIVER_PORT = 7078 private[spark] val DEFAULT_BLOCKMANAGER_PORT = 7079 - private[spark] val DEFAULT_UI_PORT = 4040 private[spark] val BLOCK_MANAGER_PORT_NAME = "blockmanager" - private[spark] val DRIVER_PORT_NAME = "driver-rpc-port" private[spark] val EXECUTOR_PORT_NAME = "executor" // Environment Variables @@ -57,20 +38,11 @@ package object constants { private[spark] val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID" private[spark] val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID" private[spark] val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP" - private[spark] val ENV_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY" - private[spark] val ENV_SUBMIT_EXTRA_CLASSPATH = "SPARK_SUBMIT_EXTRA_CLASSPATH" private[spark] val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH" private[spark] val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH" - private[spark] val ENV_DRIVER_MAIN_CLASS = "SPARK_DRIVER_CLASS" - private[spark] val ENV_DRIVER_ARGS = "SPARK_DRIVER_ARGS" - private[spark] val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS" - private[spark] val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR" - private[spark] val ENV_PYSPARK_FILES = "PYSPARK_FILES" - private[spark] val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" private[spark] val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_" // Miscellaneous - private[spark] val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" private[spark] val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" private[spark] val MEMORY_OVERHEAD_FACTOR = 0.10 private[spark] val MEMORY_OVERHEAD_MIN_MIB = 384L diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 9870d1cd61d1b..9c2c7da8cf224 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -18,8 +18,7 @@ package org.apache.spark.scheduler.cluster.k8s import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} -import org.apache.commons.io.FilenameUtils +import io.fabric8.kubernetes.api.model._ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.ConfigurationUtils @@ -48,7 +47,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) - private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs ( + private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX, "executor label") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 034180125efb4..386562fb9b3fd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -21,13 +21,12 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.SparkContext -import org.apache.spark.deploy.k8s.{ConfigurationUtils, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging -import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.ThreadUtils private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 8a53606c89fd3..8ddfd61b98062 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -21,8 +21,8 @@ import java.net.InetAddress import java.util.concurrent.{ConcurrentHashMap, ExecutorService, ScheduledExecutorService, TimeUnit} import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} -import scala.collection.{concurrent, mutable} import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import io.fabric8.kubernetes.api.model._ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index b3954f88f1457..03756a739c47e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -18,20 +18,14 @@ package org.apache.spark.scheduler.cluster.k8s import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{Pod, VolumeBuilder, VolumeMountBuilder, _} +import io.fabric8.kubernetes.api.model.{Pod, _} import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.commons.io.FilenameUtils -import org.mockito.{AdditionalAnswers, MockitoAnnotations} -import org.mockito.Matchers.{any, eq => mockitoEq} -import org.mockito.Mockito._ -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer +import org.mockito.MockitoAnnotations import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants -import org.apache.spark.scheduler.cluster.k8s.ExecutorPodFactoryImpl class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { private val driverPodName: String = "driver-pod" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 6328a781056cb..fc67f1e8fd69e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -27,7 +27,7 @@ import io.fabric8.kubernetes.client.Watcher.Action import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource} import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations} import org.mockito.Matchers.{any, eq => mockitoEq} -import org.mockito.Mockito.{mock => _, _} +import org.mockito.Mockito.{doNothing, never, times, verify, when} import org.scalatest.BeforeAndAfter import org.scalatest.mock.MockitoSugar._ @@ -74,7 +74,7 @@ private[spark] class KubernetesClusterSchedulerBackendSuite .build() private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] - private type LABELLED_PODS = FilterWatchListDeletable[ + private type LABELED_PODS = FilterWatchListDeletable[ Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]] private type IN_NAMESPACE_PODS = NonNamespaceOperation[ Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] @@ -104,7 +104,7 @@ private[spark] class KubernetesClusterSchedulerBackendSuite private var podOperations: PODS = _ @Mock - private var podsWithLabelOperations: LABELLED_PODS = _ + private var podsWithLabelOperations: LABELED_PODS = _ @Mock private var podsInNamespace: IN_NAMESPACE_PODS = _ From 82b79a7361950ffa6ca545f55628cabeb1b52049 Mon Sep 17 00:00:00 2001 From: foxish Date: Wed, 18 Oct 2017 09:26:36 -0700 Subject: [PATCH 05/29] Fix some style concerns --- .../spark/deploy/k8s/ConfigurationUtils.scala | 12 +- .../k8s/SparkKubernetesClientFactory.scala | 52 ++++---- .../cluster/k8s/ExecutorPodFactory.scala | 64 +++++----- .../k8s/KubernetesClusterManager.scala | 28 ++--- .../KubernetesClusterSchedulerBackend.scala | 32 ++--- ...bernetesClusterSchedulerBackendSuite.scala | 117 +++++++++--------- 6 files changed, 154 insertions(+), 151 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index 23832e0850c10..6d443d1774107 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -35,16 +35,18 @@ private[spark] object ConfigurationUtils { } def requireBothOrNeitherDefined( - opt1: Option[_], - opt2: Option[_], - errMessageWhenFirstIsMissing: String, - errMessageWhenSecondIsMissing: String): Unit = { + opt1: Option[_], + opt2: Option[_], + errMessageWhenFirstIsMissing: String, + errMessageWhenSecondIsMissing: String): Unit = { requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) } def requireSecondIfFirstIsDefined( - opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { + opt1: Option[_], + opt2: Option[_], + errMessageWhenSecondIsMissing: String): Unit = { opt1.foreach { _ => require(opt2.isDefined, errMessageWhenSecondIsMissing) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 864c26959f929..444144b69c341 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -49,38 +49,38 @@ private[spark] object SparkKubernetesClientFactory { .orElse(maybeServiceAccountToken) val oauthTokenValue = sparkConf.getOption(oauthTokenConf) ConfigurationUtils.requireNandDefined( - oauthTokenFile, - oauthTokenValue, - s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" + - s" value $oauthTokenConf.") + oauthTokenFile, + oauthTokenValue, + s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" + + s" value $oauthTokenConf.") val caCertFile = sparkConf - .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX") - .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath)) + .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX") + .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath)) val clientKeyFile = sparkConf - .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") + .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") val clientCertFile = sparkConf - .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") + .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") val dispatcher = new Dispatcher( - ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) + ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) val config = new ConfigBuilder() - .withApiVersion("v1") - .withMasterUrl(master) - .withWebsocketPingInterval(0) - .withOption(oauthTokenValue) { - (token, configBuilder) => configBuilder.withOauthToken(token) - }.withOption(oauthTokenFile) { - (file, configBuilder) => - configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8)) - }.withOption(caCertFile) { - (file, configBuilder) => configBuilder.withCaCertFile(file) - }.withOption(clientKeyFile) { - (file, configBuilder) => configBuilder.withClientKeyFile(file) - }.withOption(clientCertFile) { - (file, configBuilder) => configBuilder.withClientCertFile(file) - }.withOption(namespace) { - (ns, configBuilder) => configBuilder.withNamespace(ns) - }.build() + .withApiVersion("v1") + .withMasterUrl(master) + .withWebsocketPingInterval(0) + .withOption(oauthTokenValue) { + (token, configBuilder) => configBuilder.withOauthToken(token) + }.withOption(oauthTokenFile) { + (file, configBuilder) => + configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8)) + }.withOption(caCertFile) { + (file, configBuilder) => configBuilder.withCaCertFile(file) + }.withOption(clientKeyFile) { + (file, configBuilder) => configBuilder.withClientKeyFile(file) + }.withOption(clientCertFile) { + (file, configBuilder) => configBuilder.withClientCertFile(file) + }.withOption(namespace) { + (ns, configBuilder) => configBuilder.withNamespace(ns) + }.build() val baseHttpClient = HttpClientUtils.createHttpClient(config) val httpClientWithCustomDispatcher = baseHttpClient.newBuilder() .dispatcher(dispatcher) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 9c2c7da8cf224..05abc3c17b1f7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -26,8 +26,10 @@ import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.util.Utils -// Configures executor pods. Construct one of these with a SparkConf to set up properties that are -// common across all executors. Then, pass in dynamic parameters into createExecutorPod. +/** + * Configures executor pods. Construct one of these with a SparkConf to set up properties that are + * common across all executors. Then, pass in dynamic parameters into createExecutorPod. + */ private[spark] trait ExecutorPodFactory { def createExecutorPod( executorId: String, @@ -44,52 +46,52 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) import ExecutorPodFactoryImpl._ private val executorExtraClasspath = sparkConf.get( - org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) + org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( - sparkConf, - KUBERNETES_EXECUTOR_LABEL_PREFIX, - "executor label") + sparkConf, + KUBERNETES_EXECUTOR_LABEL_PREFIX, + "executor label") require( - !executorLabels.contains(SPARK_APP_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.") + !executorLabels.contains(SPARK_APP_ID_LABEL), + s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.") require( - !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + - s" Spark.") + !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL), + s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + + s" Spark.") private val executorAnnotations = - ConfigurationUtils.parsePrefixedKeyValuePairs ( - sparkConf, - KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, - "executor annotation") + ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, + KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, + "executor annotation") private val nodeSelector = - ConfigurationUtils.parsePrefixedKeyValuePairs( - sparkConf, - KUBERNETES_NODE_SELECTOR_PREFIX, - "node selector") + ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, + KUBERNETES_NODE_SELECTOR_PREFIX, + "node selector") private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE) private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) private val blockmanagerPort = sparkConf - .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) + .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) private val kubernetesDriverPodName = sparkConf - .get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(throw new SparkException("Must specify the driver pod name")) + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(throw new SparkException("Must specify the driver pod name")) private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY) private val executorMemoryString = sparkConf.get( - org.apache.spark.internal.config.EXECUTOR_MEMORY.key, - org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString) + org.apache.spark.internal.config.EXECUTOR_MEMORY.key, + org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString) private val memoryOverheadMiB = sparkConf - .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) - .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt, - MEMORY_OVERHEAD_MIN_MIB)) + .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt, + MEMORY_OVERHEAD_MIN_MIB)) private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d) @@ -109,10 +111,10 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) // executorId and applicationId val hostname = name.substring(Math.max(0, name.length - 63)) val resolvedExecutorLabels = Map( - SPARK_EXECUTOR_ID_LABEL -> executorId, - SPARK_APP_ID_LABEL -> applicationId, - SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ - executorLabels + SPARK_EXECUTOR_ID_LABEL -> executorId, + SPARK_APP_ID_LABEL -> applicationId, + SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ + executorLabels val executorMemoryQuantity = new QuantityBuilder(false) .withAmount(s"${executorMemoryMiB}Mi") .build() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 386562fb9b3fd..74aac6d211f40 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -41,25 +41,25 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val sparkConf = sc.getConf val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( - KUBERNETES_MASTER_INTERNAL_URL, - Some(sparkConf.get(KUBERNETES_NAMESPACE)), - APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX, - sparkConf, - Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), - Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) + KUBERNETES_MASTER_INTERNAL_URL, + Some(sparkConf.get(KUBERNETES_NAMESPACE)), + APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX, + sparkConf, + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) val executorPodFactory = new ExecutorPodFactoryImpl(sparkConf) val allocatorExecutor = ThreadUtils - .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") + .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( - "kubernetes-executor-requests") + "kubernetes-executor-requests") new KubernetesClusterSchedulerBackend( - scheduler.asInstanceOf[TaskSchedulerImpl], - sc.env.rpcEnv, - executorPodFactory, - kubernetesClient, - allocatorExecutor, - requestExecutorsService) + scheduler.asInstanceOf[TaskSchedulerImpl], + sc.env.rpcEnv, + executorPodFactory, + kubernetesClient, + allocatorExecutor, + requestExecutorsService) } override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 8ddfd61b98062..3afc1c60d7ebb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -65,7 +65,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .getOrElse( throw new SparkException("Must specify the driver pod name")) private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( - requestExecutorsService) + requestExecutorsService) private val driverPod = try { kubernetesClient.pods() @@ -89,9 +89,9 @@ private[spark] class KubernetesClusterSchedulerBackend( protected var totalExpectedExecutors = new AtomicInteger(0) private val driverUrl = RpcEndpointAddress( - conf.get("spark.driver.host"), - conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), - CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + conf.get("spark.driver.host"), + conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString private val initialExecutors = getInitialTargetExecutorNumber() @@ -121,7 +121,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } else { val nodeToLocalTaskCount = getNodesWithLocalTaskCounts for (i <- 0 until math.min( - totalExpectedExecutors.get - runningExecutorsToPods.size, podAllocationSize)) { + totalExpectedExecutors.get - runningExecutorsToPods.size, podAllocationSize)) { val (executorId, pod) = allocateNewExecutorPod(nodeToLocalTaskCount) runningExecutorsToPods.put(executorId, pod) runningPodsToExecutors.put(pod.getMetadata.getName, executorId) @@ -202,13 +202,13 @@ private[spark] class KubernetesClusterSchedulerBackend( override def start(): Unit = { super.start() executorWatchResource.set( - kubernetesClient - .pods() - .withLabel(SPARK_APP_ID_LABEL, applicationId()) - .watch(new ExecutorPodsWatcher())) + kubernetesClient + .pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId()) + .watch(new ExecutorPodsWatcher())) allocatorExecutor.scheduleWithFixedDelay( - allocatorRunnable, 0L, podAllocationInterval, TimeUnit.SECONDS) + allocatorRunnable, 0L, podAllocationInterval, TimeUnit.SECONDS) if (!Utils.isDynamicAllocationEnabled(conf)) { doRequestTotalExecutors(initialExecutors) @@ -281,12 +281,12 @@ private[spark] class KubernetesClusterSchedulerBackend( private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = { val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString val executorPod = executorPodFactory.createExecutorPod( - executorId, - applicationId(), - driverUrl, - conf.getExecutorEnv, - driverPod, - nodeToLocalTaskCount) + executorId, + applicationId(), + driverUrl, + conf.getExecutorEnv, + driverPod, + nodeToLocalTaskCount) try { (executorId, kubernetesClient.pods.create(executorPod)) } catch { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index fc67f1e8fd69e..6c2a6b8aef2bf 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -138,12 +138,12 @@ private[spark] class KubernetesClusterSchedulerBackendSuite before { MockitoAnnotations.initMocks(this) sparkConf = new SparkConf() - .set("spark.app.id", APP_ID) - .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) - .set(KUBERNETES_NAMESPACE, NAMESPACE) - .set("spark.driver.host", SPARK_DRIVER_HOST) - .set("spark.driver.port", SPARK_DRIVER_PORT.toString) - .set(KUBERNETES_ALLOCATION_BATCH_DELAY, POD_ALLOCATION_INTERVAL) + .set("spark.app.id", APP_ID) + .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) + .set(KUBERNETES_NAMESPACE, NAMESPACE) + .set("spark.driver.host", SPARK_DRIVER_HOST) + .set("spark.driver.port", SPARK_DRIVER_PORT.toString) + .set(KUBERNETES_ALLOCATION_BATCH_DELAY, POD_ALLOCATION_INTERVAL) executorPodsWatcherArgument = ArgumentCaptor.forClass(classOf[Watcher[Pod]]) allocatorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) requestExecutorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) @@ -154,21 +154,21 @@ private[spark] class KubernetesClusterSchedulerBackendSuite when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withLabel(SPARK_APP_ID_LABEL, APP_ID)).thenReturn(podsWithLabelOperations) when(podsWithLabelOperations.watch(executorPodsWatcherArgument.capture())) - .thenReturn(executorPodsWatch) + .thenReturn(executorPodsWatch) when(podOperations.inNamespace(NAMESPACE)).thenReturn(podsInNamespace) when(podsInNamespace.withName(DRIVER_POD_NAME)).thenReturn(podsWithDriverName) when(podsWithDriverName.get()).thenReturn(driverPod) when(allocatorExecutor.scheduleWithFixedDelay( - allocatorRunnable.capture(), - mockitoEq(0L), - mockitoEq(POD_ALLOCATION_INTERVAL), - mockitoEq(TimeUnit.SECONDS))).thenReturn(null) + allocatorRunnable.capture(), + mockitoEq(0L), + mockitoEq(POD_ALLOCATION_INTERVAL), + mockitoEq(TimeUnit.SECONDS))).thenReturn(null) // Creating Futures in Scala backed by a Java executor service resolves to running // ExecutorService#execute (as opposed to submit) doNothing().when(requestExecutorsService).execute(requestExecutorRunnable.capture()) when(rpcEnv.setupEndpoint( - mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) - .thenReturn(driverEndpointRef) + mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) + .thenReturn(driverEndpointRef) when(driverEndpointRef.ask[Boolean] (any(classOf[Any])) (any())).thenReturn(mock[Future[Boolean]]) @@ -185,8 +185,8 @@ private[spark] class KubernetesClusterSchedulerBackendSuite test("Static allocation should request executors upon first allocator run.") { sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) val scheduler = newSchedulerBackend() scheduler.start() requestExecutorRunnable.getValue.run() @@ -200,15 +200,15 @@ private[spark] class KubernetesClusterSchedulerBackendSuite test("Killing executors deletes the executor pods") { sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) val scheduler = newSchedulerBackend() scheduler.start() requestExecutorRunnable.getValue.run() expectPodCreationWithId(1, FIRST_EXECUTOR_POD) expectPodCreationWithId(2, SECOND_EXECUTOR_POD) when(podOperations.create(any(classOf[Pod]))) - .thenAnswer(AdditionalAnswers.returnsFirstArg()) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) allocatorRunnable.getValue.run() scheduler.doKillExecutors(Seq("2")) requestExecutorRunnable.getAllValues.asScala.last.run() @@ -218,8 +218,8 @@ private[spark] class KubernetesClusterSchedulerBackendSuite test("Executors should be requested in batches.") { sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) val scheduler = newSchedulerBackend() scheduler.start() requestExecutorRunnable.getValue.run() @@ -231,10 +231,10 @@ private[spark] class KubernetesClusterSchedulerBackendSuite verify(podOperations).create(FIRST_EXECUTOR_POD) verify(podOperations, never()).create(SECOND_EXECUTOR_POD) val registerFirstExecutorMessage = RegisterExecutor( - "1", mock[RpcEndpointRef], "localhost", 1, Map.empty[String, String]) + "1", mock[RpcEndpointRef], "localhost", 1, Map.empty[String, String]) when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) - .apply(registerFirstExecutorMessage) + .apply(registerFirstExecutorMessage) allocatorRunnable.getValue.run() verify(podOperations).create(SECOND_EXECUTOR_POD) } @@ -242,22 +242,22 @@ private[spark] class KubernetesClusterSchedulerBackendSuite test("Deleting executors and then running an allocator pass after finding the loss reason" + " should only delete the pod once.") { sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) val scheduler = newSchedulerBackend() scheduler.start() requestExecutorRunnable.getValue.run() when(podOperations.create(any(classOf[Pod]))) - .thenAnswer(AdditionalAnswers.returnsFirstArg()) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) expectPodCreationWithId(1, FIRST_EXECUTOR_POD) allocatorRunnable.getValue.run() val executorEndpointRef = mock[RpcEndpointRef] when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) val registerFirstExecutorMessage = RegisterExecutor( - "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) - .apply(registerFirstExecutorMessage) + .apply(registerFirstExecutorMessage) scheduler.doRequestTotalExecutors(0) requestExecutorRunnable.getAllValues.asScala.last.run() scheduler.doKillExecutors(Seq("1")) @@ -270,17 +270,17 @@ private[spark] class KubernetesClusterSchedulerBackendSuite allocatorRunnable.getValue.run() verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) verify(driverEndpointRef, times(1)).ask[Boolean]( - RemoveExecutor("1", ExecutorExited( - 0, - exitCausedByApp = false, - s"Container in pod ${exitedPod.getMetadata.getName} exited from" + - s" explicit termination request."))) + RemoveExecutor("1", ExecutorExited( + 0, + exitCausedByApp = false, + s"Container in pod ${exitedPod.getMetadata.getName} exited from" + + s" explicit termination request."))) } test("Executors that disconnect from application errors are noted as exits caused by app.") { sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) val scheduler = newSchedulerBackend() scheduler.start() expectPodCreationWithId(1, FIRST_EXECUTOR_POD) @@ -290,32 +290,32 @@ private[spark] class KubernetesClusterSchedulerBackendSuite val executorEndpointRef = mock[RpcEndpointRef] when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) val registerFirstExecutorMessage = RegisterExecutor( - "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) - .apply(registerFirstExecutorMessage) + .apply(registerFirstExecutorMessage) driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) executorPodsWatcherArgument.getValue.eventReceived( - Action.ERROR, exitPod(FIRST_EXECUTOR_POD, 1)) + Action.ERROR, exitPod(FIRST_EXECUTOR_POD, 1)) expectPodCreationWithId(2, SECOND_EXECUTOR_POD) scheduler.doRequestTotalExecutors(1) requestExecutorRunnable.getValue.run() allocatorRunnable.getAllValues.asScala.last.run() verify(driverEndpointRef).ask[Boolean]( - RemoveExecutor("1", ExecutorExited( - 1, - exitCausedByApp = true, - s"Pod ${FIRST_EXECUTOR_POD.getMetadata.getName}'s executor container exited with" + - " exit status code 1."))) + RemoveExecutor("1", ExecutorExited( + 1, + exitCausedByApp = true, + s"Pod ${FIRST_EXECUTOR_POD.getMetadata.getName}'s executor container exited with" + + " exit status code 1."))) verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) } test("Executors should only try to get the loss reason a number of times before giving up and" + " removing the executor.") { sparkConf - .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) - .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) val scheduler = newSchedulerBackend() scheduler.start() expectPodCreationWithId(1, FIRST_EXECUTOR_POD) @@ -325,10 +325,10 @@ private[spark] class KubernetesClusterSchedulerBackendSuite val executorEndpointRef = mock[RpcEndpointRef] when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) val registerFirstExecutorMessage = RegisterExecutor( - "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) - .apply(registerFirstExecutorMessage) + .apply(registerFirstExecutorMessage) driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) 1 to KubernetesClusterSchedulerBackend.MAX_EXECUTOR_LOST_REASON_CHECKS foreach { _ => allocatorRunnable.getValue.run() @@ -343,12 +343,12 @@ private[spark] class KubernetesClusterSchedulerBackendSuite private def newSchedulerBackend(): KubernetesClusterSchedulerBackend = { new KubernetesClusterSchedulerBackend( - taskSchedulerImpl, - rpcEnv, - executorPodFactory, - kubernetesClient, - allocatorExecutor, - requestExecutorsService) + taskSchedulerImpl, + rpcEnv, + executorPodFactory, + kubernetesClient, + allocatorExecutor, + requestExecutorsService) } private def exitPod(basePod: Pod, exitCode: Int): Pod = { @@ -367,12 +367,11 @@ private[spark] class KubernetesClusterSchedulerBackendSuite private def expectPodCreationWithId(executorId: Int, expectedPod: Pod): Unit = { when(executorPodFactory.createExecutorPod( - executorId.toString, - APP_ID, - DRIVER_URL, - sparkConf.getExecutorEnv, - driverPod, - Map.empty)).thenReturn(expectedPod) + executorId.toString, + APP_ID, + DRIVER_URL, + sparkConf.getExecutorEnv, + driverPod, + Map.empty)).thenReturn(expectedPod) } - } From c052212888e01eac90a006bfb5d14c513e33d0a3 Mon Sep 17 00:00:00 2001 From: foxish Date: Fri, 20 Oct 2017 10:54:45 -0700 Subject: [PATCH 06/29] Clean up YARN constants, unit test updates --- .../KubernetesClusterSchedulerBackend.scala | 21 ++---------- ...bernetesClusterSchedulerBackendSuite.scala | 34 +++++++++++++++---- 2 files changed, 30 insertions(+), 25 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 3afc1c60d7ebb..05d93fda3c740 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -390,16 +390,8 @@ private[spark] class KubernetesClusterSchedulerBackend( s"Container in pod " + pod.getMetadata.getName + " exited from explicit termination request.") } else { - val containerExitReason = containerExitStatus match { - case VMEM_EXCEEDED_EXIT_CODE | PMEM_EXCEEDED_EXIT_CODE => - memLimitExceededLogMessage(pod.getStatus.getReason) - case _ => - // Here we can't be sure that that exit was caused by the application but this seems - // to be the right default since we know the pod was not explicitly deleted by - // the user. - s"Pod ${pod.getMetadata.getName}'s executor container exited with exit status" + - s" code $containerExitStatus." - } + val containerExitReason = s"Pod ${pod.getMetadata.getName}'s executor container " + + s"exited with exit status code $containerExitStatus." ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason) } podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason) @@ -441,16 +433,9 @@ private[spark] class KubernetesClusterSchedulerBackend( } private object KubernetesClusterSchedulerBackend { - private val VMEM_EXCEEDED_EXIT_CODE = -103 - private val PMEM_EXCEEDED_EXIT_CODE = -104 - private val UNKNOWN_EXIT_CODE = -111 + private val UNKNOWN_EXIT_CODE = -1 // Number of times we are allowed check for the loss reason for an executor before we give up // and assume the executor failed for good, and attribute it to a framework fault. val MAX_EXECUTOR_LOST_REASON_CHECKS = 10 - - def memLimitExceededLogMessage(diagnostics: String): String = { - s"Pod/Container killed for exceeding memory limits. $diagnostics" + - " Consider boosting spark executor memory overhead." - } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 6c2a6b8aef2bf..38aaa20ead711 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -38,8 +38,9 @@ import org.apache.spark.rpc._ import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.ThreadUtils -private[spark] class KubernetesClusterSchedulerBackendSuite +class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAndAfter { private val APP_ID = "test-spark-app" @@ -121,6 +122,9 @@ private[spark] class KubernetesClusterSchedulerBackendSuite @Mock private var executorPodsWatch: Watch = _ + @Mock + private var successFuture: Future[Boolean] = _ + private var sparkConf: SparkConf = _ private var executorPodsWatcherArgument: ArgumentCaptor[Watcher[Pod]] = _ private var allocatorRunnable: ArgumentCaptor[Runnable] = _ @@ -169,9 +173,15 @@ private[spark] class KubernetesClusterSchedulerBackendSuite when(rpcEnv.setupEndpoint( mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) .thenReturn(driverEndpointRef) + + // Used by the CoarseGrainedSchedulerBackend when making RPC calls. when(driverEndpointRef.ask[Boolean] (any(classOf[Any])) - (any())).thenReturn(mock[Future[Boolean]]) + (any())).thenReturn(successFuture) + when(successFuture.failed).thenReturn(Future[Throwable] { + // emulate behavior of the Future.failed method. + throw new NoSuchElementException() + }(ThreadUtils.sameThread)) } test("Basic lifecycle expectations when starting and stopping the scheduler.") { @@ -239,13 +249,14 @@ private[spark] class KubernetesClusterSchedulerBackendSuite verify(podOperations).create(SECOND_EXECUTOR_POD) } - test("Deleting executors and then running an allocator pass after finding the loss reason" + - " should only delete the pod once.") { + test("Scaled down executors should be cleaned up") { sparkConf .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) val scheduler = newSchedulerBackend() scheduler.start() + + // The scheduler backend spins up one executor pod. requestExecutorRunnable.getValue.run() when(podOperations.create(any(classOf[Pod]))) .thenAnswer(AdditionalAnswers.returnsFirstArg()) @@ -258,6 +269,8 @@ private[spark] class KubernetesClusterSchedulerBackendSuite when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) .apply(registerFirstExecutorMessage) + + // Request that there are 0 executors and trigger deletion from driver. scheduler.doRequestTotalExecutors(0) requestExecutorRunnable.getAllValues.asScala.last.run() scheduler.doKillExecutors(Seq("1")) @@ -268,6 +281,9 @@ private[spark] class KubernetesClusterSchedulerBackendSuite val exitedPod = exitPod(FIRST_EXECUTOR_POD, 0) executorPodsWatcherArgument.getValue.eventReceived(Action.DELETED, exitedPod) allocatorRunnable.getValue.run() + + // No more deletion attempts of the executors. + // This is graceful termination and should not be detected as a failure. verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) verify(driverEndpointRef, times(1)).ask[Boolean]( RemoveExecutor("1", ExecutorExited( @@ -277,10 +293,11 @@ private[spark] class KubernetesClusterSchedulerBackendSuite s" explicit termination request."))) } - test("Executors that disconnect from application errors are noted as exits caused by app.") { + test("Executors that fail should not be deleted.") { sparkConf .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend() scheduler.start() expectPodCreationWithId(1, FIRST_EXECUTOR_POD) @@ -298,6 +315,7 @@ private[spark] class KubernetesClusterSchedulerBackendSuite executorPodsWatcherArgument.getValue.eventReceived( Action.ERROR, exitPod(FIRST_EXECUTOR_POD, 1)) + // A replacement executor should be created but the error pod should persist. expectPodCreationWithId(2, SECOND_EXECUTOR_POD) scheduler.doRequestTotalExecutors(1) requestExecutorRunnable.getValue.run() @@ -311,11 +329,11 @@ private[spark] class KubernetesClusterSchedulerBackendSuite verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) } - test("Executors should only try to get the loss reason a number of times before giving up and" + - " removing the executor.") { + test("Executors disconnected due to unknown reasons are deleted and replaced.") { sparkConf .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend() scheduler.start() expectPodCreationWithId(1, FIRST_EXECUTOR_POD) @@ -329,11 +347,13 @@ private[spark] class KubernetesClusterSchedulerBackendSuite when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) .apply(registerFirstExecutorMessage) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) 1 to KubernetesClusterSchedulerBackend.MAX_EXECUTOR_LOST_REASON_CHECKS foreach { _ => allocatorRunnable.getValue.run() verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) } + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) allocatorRunnable.getValue.run() verify(podOperations).delete(FIRST_EXECUTOR_POD) From c565c9ffd7e5371ee4425d69ecaf49ce92199fc7 Mon Sep 17 00:00:00 2001 From: foxish Date: Fri, 20 Oct 2017 14:37:17 -0700 Subject: [PATCH 07/29] Couple of more style comments --- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 05d93fda3c740..9d6efec171625 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -62,8 +62,7 @@ private[spark] class KubernetesClusterSchedulerBackend( private val kubernetesDriverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse( - throw new SparkException("Must specify the driver pod name")) + .getOrElse(throw new SparkException("Must specify the driver pod name")) private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( requestExecutorsService) @@ -86,7 +85,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } private val executorWatchResource = new AtomicReference[Closeable] - protected var totalExpectedExecutors = new AtomicInteger(0) + protected val totalExpectedExecutors = new AtomicInteger(0) private val driverUrl = RpcEndpointAddress( conf.get("spark.driver.host"), From 2fb596d024b4115c9209df6e9fae24550e08a89b Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 25 Oct 2017 12:58:32 -0700 Subject: [PATCH 08/29] Address CR comments. --- .../org/apache/spark/deploy/k8s/config.scala | 6 ++-- .../KubernetesClusterSchedulerBackend.scala | 33 ++++++++----------- 2 files changed, 17 insertions(+), 22 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala index 5c66a125bf114..a85607815dfde 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala @@ -93,13 +93,13 @@ package object config extends Logging { private[spark] val KUBERNETES_ALLOCATION_BATCH_SIZE = ConfigBuilder("spark.kubernetes.allocation.batch.size") - .doc("Number of pods to launch at once in each round of dynamic allocation. ") + .doc("Number of pods to launch at once in each round of executor allocation.") .intConf .createWithDefault(5) private[spark] val KUBERNETES_ALLOCATION_BATCH_DELAY = ConfigBuilder("spark.kubernetes.allocation.batch.delay") - .doc("Number of seconds to wait between each round of executor allocation. ") + .doc("Number of seconds to wait between each round of executor allocation.") .longConf .createWithDefault(1) @@ -129,7 +129,7 @@ package object config extends Logging { masterWithoutK8sPrefix } else { val resolvedURL = s"https://$masterWithoutK8sPrefix" - logDebug(s"No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + + logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + s" URL is $resolvedURL") resolvedURL } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 9d6efec171625..4182c1622ee23 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -66,16 +66,10 @@ private[spark] class KubernetesClusterSchedulerBackend( private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( requestExecutorsService) - private val driverPod = try { - kubernetesClient.pods() - .inNamespace(kubernetesNamespace) - .withName(kubernetesDriverPodName) - .get() - } catch { - case throwable: Throwable => - logError(s"Executor cannot find driver pod.", throwable) - throw new SparkException(s"Executor cannot find driver pod", throwable) - } + private val driverPod = kubernetesClient.pods() + .inNamespace(kubernetesNamespace) + .withName(kubernetesDriverPodName) + .get() override val minRegisteredRatio = if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { @@ -142,13 +136,16 @@ private[spark] class KubernetesClusterSchedulerBackend( knownExitReason.fold { removeExecutorOrIncrementLossReasonCheckCount(executorId) } { executorExited => - logDebug(s"Removing executor $executorId with loss reason " + executorExited.message) + logWarning(s"Removing executor $executorId with loss reason " + executorExited.message) removeExecutor(executorId, executorExited) // We keep around executors that have exit conditions caused by the application. This // allows them to be debugged later on. Otherwise, mark them as to be deleted from the // the API server. if (!executorExited.exitCausedByApp) { + logInfo(s"Executor $executorId failed because of a framework error.") deleteExecutorFromClusterAndDataStructures(executorId) + } else { + logInfo(s"Executor $executorId exited because of the application.") } } } @@ -192,8 +189,6 @@ private[spark] class KubernetesClusterSchedulerBackend( } - override def applicationId(): String = conf.get("spark.app.id", super.applicationId()) - override def sufficientResourcesRegistered(): Boolean = { totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio } @@ -331,10 +326,10 @@ private[spark] class KubernetesClusterSchedulerBackend( override def eventReceived(action: Action, pod: Pod): Unit = { if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running" - && pod.getMetadata.getDeletionTimestamp == null) { + && pod.getMetadata.getDeletionTimestamp == null) { val podIP = pod.getStatus.getPodIP val clusterNodeName = pod.getSpec.getNodeName - logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") + logInfo(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") executorPodsByIPs.put(podIP, pod) } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) || action == Action.DELETED || action == Action.ERROR) { @@ -345,10 +340,10 @@ private[spark] class KubernetesClusterSchedulerBackend( executorPodsByIPs.remove(podIP) } if (action == Action.ERROR) { - logInfo(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) + logWarning(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) handleErroredPod(pod) } else if (action == Action.DELETED) { - logInfo(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason) + logWarning(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason) handleDeletedPod(pod) } } @@ -386,8 +381,8 @@ private[spark] class KubernetesClusterSchedulerBackend( // container was probably actively killed by the driver. val exitReason = if (isPodAlreadyReleased(pod)) { ExecutorExited(containerExitStatus, exitCausedByApp = false, - s"Container in pod " + pod.getMetadata.getName + - " exited from explicit termination request.") + s"Container in pod ${pod.getMetadata.getName} exited from explicit termination" + + " request.") } else { val containerExitReason = s"Pod ${pod.getMetadata.getName}'s executor container " + s"exited with exit status code $containerExitStatus." From 992acbee3730e966543832ed08318e4ed1424064 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 25 Oct 2017 13:36:26 -0700 Subject: [PATCH 09/29] Extract initial executor count to utils class --- .../cluster/SchedulerBackendUtils.scala | 47 +++++++++++++++++++ .../KubernetesClusterSchedulerBackend.scala | 21 +-------- .../spark/deploy/yarn/YarnAllocator.scala | 4 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 22 --------- .../cluster/YarnClientSchedulerBackend.scala | 2 +- .../cluster/YarnClusterSchedulerBackend.scala | 2 +- 6 files changed, 53 insertions(+), 45 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackendUtils.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackendUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackendUtils.scala new file mode 100644 index 0000000000000..d6ea5c277b4ee --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackendUtils.scala @@ -0,0 +1,47 @@ +/* + * 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.scheduler.cluster + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.{DYN_ALLOCATION_MAX_EXECUTORS, DYN_ALLOCATION_MIN_EXECUTORS, EXECUTOR_INSTANCES} +import org.apache.spark.util.Utils + +private[spark] object SchedulerBackendUtils { + val DEFAULT_NUMBER_EXECUTORS = 2 + + /** + * Getting the initial target number of executors depends on whether dynamic allocation is + * enabled. + * If not using dynamic allocation it gets the number of executors requested by the user. + */ + def getInitialTargetExecutorNumber( + conf: SparkConf, + numExecutors: Int = DEFAULT_NUMBER_EXECUTORS): Int = { + if (Utils.isDynamicAllocationEnabled(conf)) { + val minNumExecutors = conf.get(DYN_ALLOCATION_MIN_EXECUTORS) + val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) + val maxNumExecutors = conf.get(DYN_ALLOCATION_MAX_EXECUTORS) + require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, + s"initial executor number $initialNumExecutors must between min executor number " + + s"$minNumExecutors and max executor number $maxNumExecutors") + + initialNumExecutors + } else { + conf.get(EXECUTOR_INSTANCES).getOrElse(numExecutors) + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 4182c1622ee23..a538993dfdc3d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} - import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action @@ -34,7 +33,7 @@ import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} import org.apache.spark.util.Utils private[spark] class KubernetesClusterSchedulerBackend( @@ -86,7 +85,7 @@ private[spark] class KubernetesClusterSchedulerBackend( conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString - private val initialExecutors = getInitialTargetExecutorNumber() + private val initialExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(conf) private val podAllocationInterval = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) require(podAllocationInterval > 0, s"Allocation batch delay " + @@ -173,22 +172,6 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - private def getInitialTargetExecutorNumber(defaultNumExecutors: Int = 1): Int = { - if (Utils.isDynamicAllocationEnabled(conf)) { - val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) - val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) - val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", 1) - require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, - s"initial executor number $initialNumExecutors must between min executor number " + - s"$minNumExecutors and max executor number $maxNumExecutors") - - initialNumExecutors - } else { - conf.getInt("spark.executor.instances", defaultNumExecutors) - } - - } - override def sufficientResourcesRegistered(): Boolean = { totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 7052fb347106b..6e8f2b4aa628e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -26,7 +26,6 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} import scala.util.control.NonFatal - import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest @@ -41,6 +40,7 @@ import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RetrieveLastAllocatedExecutorId +import org.apache.spark.scheduler.cluster.SchedulerBackendUtils import org.apache.spark.util.{Clock, SystemClock, ThreadUtils} /** @@ -109,7 +109,7 @@ private[yarn] class YarnAllocator( sparkConf.get(EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS).getOrElse(-1L) @volatile private var targetNumExecutors = - YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf) + SchedulerBackendUtils.getInitialTargetExecutorNumber(sparkConf) private var currentNodeBlacklist = Set.empty[String] diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 3d9f99f57bed7..ca6f62c517604 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -279,27 +279,5 @@ object YarnSparkHadoopUtil { securityMgr.getModifyAclsGroups) ) } - - /** - * Getting the initial target number of executors depends on whether dynamic allocation is - * enabled. - * If not using dynamic allocation it gets the number of executors requested by the user. - */ - def getInitialTargetExecutorNumber( - conf: SparkConf, - numExecutors: Int = DEFAULT_NUMBER_EXECUTORS): Int = { - if (Utils.isDynamicAllocationEnabled(conf)) { - val minNumExecutors = conf.get(DYN_ALLOCATION_MIN_EXECUTORS) - val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) - val maxNumExecutors = conf.get(DYN_ALLOCATION_MAX_EXECUTORS) - require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, - s"initial executor number $initialNumExecutors must between min executor number " + - s"$minNumExecutors and max executor number $maxNumExecutors") - - initialNumExecutors - } else { - conf.get(EXECUTOR_INSTANCES).getOrElse(numExecutors) - } - } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index d482376d14dd7..b722cc401bb73 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -52,7 +52,7 @@ private[spark] class YarnClientSchedulerBackend( logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" ")) val args = new ClientArguments(argsArrayBuf.toArray) - totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(conf) + totalExpectedExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(conf) client = new Client(args, conf) bindToYarn(client.submitApplication(), None) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 4f3d5ebf403e0..e2d477be329c3 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -34,7 +34,7 @@ private[spark] class YarnClusterSchedulerBackend( val attemptId = ApplicationMaster.getAttemptId bindToYarn(attemptId.getApplicationId(), Some(attemptId)) super.start() - totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sc.conf) + totalExpectedExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(sc.conf) } override def getDriverLogUrls: Option[Map[String, String]] = { From b0a5839684d3eb08e1ad93db735440c26b3a1d1e Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 25 Oct 2017 13:48:30 -0700 Subject: [PATCH 10/29] Fix scalastyle --- .../spark/scheduler/cluster/SchedulerBackendUtils.scala | 8 ++++---- .../org/apache/spark/deploy/yarn/YarnAllocator.scala | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackendUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackendUtils.scala index d6ea5c277b4ee..c166d030f2c89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackendUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackendUtils.scala @@ -24,10 +24,10 @@ private[spark] object SchedulerBackendUtils { val DEFAULT_NUMBER_EXECUTORS = 2 /** - * Getting the initial target number of executors depends on whether dynamic allocation is - * enabled. - * If not using dynamic allocation it gets the number of executors requested by the user. - */ + * Getting the initial target number of executors depends on whether dynamic allocation is + * enabled. + * If not using dynamic allocation it gets the number of executors requested by the user. + */ def getInitialTargetExecutorNumber( conf: SparkConf, numExecutors: Int = DEFAULT_NUMBER_EXECUTORS): Int = { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 6e8f2b4aa628e..a3f3718f245b8 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -22,14 +22,14 @@ import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import java.util.regex.Pattern -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} -import scala.util.control.NonFatal import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import scala.util.control.NonFatal import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ From a4f97976afff452d7d953b83b722da61dfb40c3b Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 25 Oct 2017 14:07:29 -0700 Subject: [PATCH 11/29] Fix more scalastyle --- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index a538993dfdc3d..4d074aae7c003 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -21,12 +21,12 @@ import java.net.InetAddress import java.util.concurrent.{ConcurrentHashMap, ExecutorService, ScheduledExecutorService, TimeUnit} import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.concurrent.{ExecutionContext, Future} import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.config._ From 2b5dcac4f188b8c61b67aec131a1eb7d91968356 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 26 Oct 2017 16:14:33 -0700 Subject: [PATCH 12/29] Pin down app ID in tests. Fix test style. --- .../k8s/KubernetesClusterSchedulerBackendSuite.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 38aaa20ead711..3ce9478fde74b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -18,9 +18,6 @@ package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.{ExecutorService, ScheduledExecutorService, TimeUnit} -import scala.collection.JavaConverters._ -import scala.concurrent.Future - import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList} import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher} import io.fabric8.kubernetes.client.Watcher.Action @@ -30,6 +27,8 @@ import org.mockito.Matchers.{any, eq => mockitoEq} import org.mockito.Mockito.{doNothing, never, times, verify, when} import org.scalatest.BeforeAndAfter import org.scalatest.mock.MockitoSugar._ +import scala.collection.JavaConverters._ +import scala.concurrent.Future import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.deploy.k8s.config._ @@ -142,7 +141,6 @@ class KubernetesClusterSchedulerBackendSuite before { MockitoAnnotations.initMocks(this) sparkConf = new SparkConf() - .set("spark.app.id", APP_ID) .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) .set(KUBERNETES_NAMESPACE, NAMESPACE) .set("spark.driver.host", SPARK_DRIVER_HOST) @@ -368,7 +366,10 @@ class KubernetesClusterSchedulerBackendSuite executorPodFactory, kubernetesClient, allocatorExecutor, - requestExecutorsService) + requestExecutorsService) { + + override def applicationId(): String = APP_ID + } } private def exitPod(basePod: Pod, exitCode: Int): Pod = { From 018f4d8ffbbe33526a8273801169b99add38fc8f Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 1 Nov 2017 13:56:00 -0700 Subject: [PATCH 13/29] Address comments. --- .../org/apache/spark/deploy/k8s/config.scala | 24 --- .../cluster/k8s/ExecutorPodFactory.scala | 10 +- .../KubernetesClusterSchedulerBackend.scala | 154 +++++++++--------- 3 files changed, 82 insertions(+), 106 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala index a85607815dfde..f49056e06554c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala @@ -103,14 +103,6 @@ package object config extends Logging { .longConf .createWithDefault(1) - private[spark] val INIT_CONTAINER_JARS_DOWNLOAD_LOCATION = - ConfigBuilder("spark.kubernetes.mountdependencies.jarsDownloadDir") - .doc("Location to download jars to in the driver and executors. When using" + - " spark-submit, this directory must be empty and will be mounted as an empty directory" + - " volume on the driver and executor pod.") - .stringConf - .createWithDefault("/var/spark-data/spark-jars") - private[spark] val KUBERNETES_EXECUTOR_LIMIT_CORES = ConfigBuilder("spark.kubernetes.executor.limit.cores") .doc("Specify the hard cpu limit for a single executor pod") @@ -118,20 +110,4 @@ package object config extends Logging { .createOptional private[spark] val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." - - private[spark] def resolveK8sMaster(rawMasterString: String): String = { - if (!rawMasterString.startsWith("k8s://")) { - throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") - } - val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "") - if (masterWithoutK8sPrefix.startsWith("http://") - || masterWithoutK8sPrefix.startsWith("https://")) { - masterWithoutK8sPrefix - } else { - val resolvedURL = s"https://$masterWithoutK8sPrefix" - logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + - s" URL is $resolvedURL") - resolvedURL - } - } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 05abc3c17b1f7..03e8e9d255349 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -47,7 +47,6 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) private val executorExtraClasspath = sparkConf.get( org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) - private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, @@ -94,7 +93,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) MEMORY_OVERHEAD_MIN_MIB)) private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB - private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d) + private val executorCores = sparkConf.getDouble("spark.executor.cores", 1) private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key) override def createExecutorPod( @@ -108,7 +107,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) // hostname must be no longer than 63 characters, so take the last 63 characters of the pod // name as the hostname. This preserves uniqueness since the end of name contains - // executorId and applicationId + // executorId val hostname = name.substring(Math.max(0, name.length - 63)) val resolvedExecutorLabels = Map( SPARK_EXECUTOR_ID_LABEL -> executorId, @@ -139,15 +138,14 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() } }.getOrElse(Seq.empty[EnvVar]) - val executorEnv = (Seq( + val executorEnv = Seq( (ENV_EXECUTOR_PORT, executorPort.toString), (ENV_DRIVER_URL, driverUrl), // Executor backend expects integral value for executor cores, so round it up to an int. (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString), (ENV_EXECUTOR_MEMORY, executorMemoryString), (ENV_APPLICATION_ID, applicationId), - (ENV_EXECUTOR_ID, executorId), - (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs) + (ENV_EXECUTOR_ID, executorId)) .map(env => new EnvVarBuilder() .withName(env._1) .withValue(env._2) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 4d074aae7c003..a567a7a67da08 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -20,6 +20,7 @@ import java.io.Closeable import java.net.InetAddress import java.util.concurrent.{ConcurrentHashMap, ExecutorService, ScheduledExecutorService, TimeUnit} import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} +import javax.annotation.concurrent.GuardedBy import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} @@ -49,9 +50,11 @@ private[spark] class KubernetesClusterSchedulerBackend( private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) private val RUNNING_EXECUTOR_PODS_LOCK = new Object - // Indexed by executor IDs and guarded by RUNNING_EXECUTOR_PODS_LOCK. + // Indexed by executor IDs + @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK") private val runningExecutorsToPods = new mutable.HashMap[String, Pod] - // Indexed by executor pod names and guarded by RUNNING_EXECUTOR_PODS_LOCK. + // Indexed by executor pod names + @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK") private val runningPodsToExecutors = new mutable.HashMap[String, String] private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]() private val podsWithKnownExitReasons = new ConcurrentHashMap[String, ExecutorExited]() @@ -105,21 +108,44 @@ private[spark] class KubernetesClusterSchedulerBackend( override def run(): Unit = { handleDisconnectedExecutors() + val executorsToAllocate = mutable.Map[String, Pod]() + val currentTotalRegisteredExecutors = totalRegisteredExecutors.get + val currentTotalExpectedExecutors = totalExpectedExecutors.get + val currentNodeToLocalTaskCount = getNodesWithLocalTaskCounts + if (currentTotalRegisteredExecutors < runningExecutorsToPods.size) { + logDebug("Waiting for pending executors before scaling") + } else if (currentTotalExpectedExecutors <= runningExecutorsToPods.size) { + logDebug("Maximum allowed executor limit reached. Not scaling up further.") + } else { + val nodeToLocalTaskCount = getNodesWithLocalTaskCounts + for (i <- 0 until math.min( + currentTotalExpectedExecutors - runningExecutorsToPods.size, podAllocationSize)) { + val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString + val executorPod = executorPodFactory.createExecutorPod( + executorId, + applicationId(), + driverUrl, + conf.getExecutorEnv, + driverPod, + nodeToLocalTaskCount) + executorsToAllocate(executorId) = executorPod + logInfo( + s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}") + } + } + val allocatedExecutors = executorsToAllocate.mapValues { pod => + Utils.tryLog { + kubernetesClient.pods().create(pod) + } + } RUNNING_EXECUTOR_PODS_LOCK.synchronized { - if (totalRegisteredExecutors.get() < runningExecutorsToPods.size) { - logDebug("Waiting for pending executors before scaling") - } else if (totalExpectedExecutors.get() <= runningExecutorsToPods.size) { - logDebug("Maximum allowed executor limit reached. Not scaling up further.") - } else { - val nodeToLocalTaskCount = getNodesWithLocalTaskCounts - for (i <- 0 until math.min( - totalExpectedExecutors.get - runningExecutorsToPods.size, podAllocationSize)) { - val (executorId, pod) = allocateNewExecutorPod(nodeToLocalTaskCount) - runningExecutorsToPods.put(executorId, pod) - runningPodsToExecutors.put(pod.getMetadata.getName, executorId) - logInfo( - s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}") - } + allocatedExecutors.map { + case (executorId, attemptedAllocatedExecutor) => + attemptedAllocatedExecutor.map { successfullyAllocatedExecutor => + runningExecutorsToPods.put(executorId, successfullyAllocatedExecutor) + runningPodsToExecutors.put( + successfullyAllocatedExecutor.getMetadata.getName, executorId) + } } } } @@ -128,25 +154,25 @@ private[spark] class KubernetesClusterSchedulerBackend( // For each disconnected executor, synchronize with the loss reasons that may have been found // by the executor pod watcher. If the loss reason was discovered by the watcher, // inform the parent class with removeExecutor. - disconnectedPodsByExecutorIdPendingRemoval.keys().asScala.foreach { case (executorId) => - val executorPod = disconnectedPodsByExecutorIdPendingRemoval.get(executorId) - val knownExitReason = Option(podsWithKnownExitReasons.remove( - executorPod.getMetadata.getName)) - knownExitReason.fold { - removeExecutorOrIncrementLossReasonCheckCount(executorId) - } { executorExited => - logWarning(s"Removing executor $executorId with loss reason " + executorExited.message) - removeExecutor(executorId, executorExited) - // We keep around executors that have exit conditions caused by the application. This - // allows them to be debugged later on. Otherwise, mark them as to be deleted from the - // the API server. - if (!executorExited.exitCausedByApp) { - logInfo(s"Executor $executorId failed because of a framework error.") - deleteExecutorFromClusterAndDataStructures(executorId) - } else { - logInfo(s"Executor $executorId exited because of the application.") + disconnectedPodsByExecutorIdPendingRemoval.asScala.foreach { + case (executorId, executorPod) => + val knownExitReason = Option(podsWithKnownExitReasons.remove( + executorPod.getMetadata.getName)) + knownExitReason.fold { + removeExecutorOrIncrementLossReasonCheckCount(executorId) + } { executorExited => + logWarning(s"Removing executor $executorId with loss reason " + executorExited.message) + removeExecutor(executorId, executorExited) + // We keep around executors that have exit conditions caused by the application. This + // allows them to be debugged later on. Otherwise, mark them as to be deleted from the + // the API server. + if (!executorExited.exitCausedByApp) { + logInfo(s"Executor $executorId failed because of a framework error.") + deleteExecutorFromClusterAndDataStructures(executorId) + } else { + logInfo(s"Executor $executorId exited because of the application.") + } } - } } } @@ -163,12 +189,17 @@ private[spark] class KubernetesClusterSchedulerBackend( def deleteExecutorFromClusterAndDataStructures(executorId: String): Unit = { disconnectedPodsByExecutorIdPendingRemoval.remove(executorId) executorReasonCheckAttemptCounts -= executorId - RUNNING_EXECUTOR_PODS_LOCK.synchronized { + podsWithKnownExitReasons -= executorId + val maybeExecutorPodToDelete = RUNNING_EXECUTOR_PODS_LOCK.synchronized { runningExecutorsToPods.remove(executorId).map { pod => - kubernetesClient.pods().delete(pod) runningPodsToExecutors.remove(pod.getMetadata.getName) - }.getOrElse(logWarning(s"Unable to remove pod for unknown executor $executorId")) + pod + }.orElse { + logWarning(s"Unable to remove pod for unknown executor $executorId") + None + } } + maybeExecutorPodToDelete.foreach(pod => kubernetesClient.pods().delete(pod)) } } @@ -203,25 +234,23 @@ private[spark] class KubernetesClusterSchedulerBackend( // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context. // When using Utils.tryLogNonFatalError some of the code fails but without any logs or // indication as to why. - try { - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_)) + Utils.tryLogNonFatalError { + val executorPodsToDelete = RUNNING_EXECUTOR_PODS_LOCK.synchronized { + val runningExecutorPodsCopy = Seq(runningExecutorsToPods.values.toSeq: _*) runningExecutorsToPods.clear() runningPodsToExecutors.clear() + runningExecutorPodsCopy } + kubernetesClient.pods().delete(executorPodsToDelete: _*) executorPodsByIPs.clear() val resource = executorWatchResource.getAndSet(null) if (resource != null) { resource.close() } - } catch { - case e: Throwable => logError("Uncaught exception while shutting down controllers.", e) } - try { + Utils.tryLogNonFatalError { logInfo("Closing kubernetes client") kubernetesClient.close() - } catch { - case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e) } } @@ -231,7 +260,7 @@ private[spark] class KubernetesClusterSchedulerBackend( */ private def getNodesWithLocalTaskCounts() : Map[String, Int] = { val nodeToLocalTaskCount = mutable.Map[String, Int]() ++ - KubernetesClusterSchedulerBackend.this.synchronized { + synchronized { hostToLocalTaskCount } for (pod <- executorPodsByIPs.values().asScala) { @@ -247,58 +276,31 @@ private[spark] class KubernetesClusterSchedulerBackend( nodeToLocalTaskCount.toMap[String, Int] } - /** - * Allocates a new executor pod - * - * @param nodeToLocalTaskCount A map of K8s cluster nodes to the number of tasks that could - * benefit from data locality if an executor launches on the cluster - * node. - * @return A tuple of the new executor name and the Pod data structure. - */ - private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = { - val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString - val executorPod = executorPodFactory.createExecutorPod( - executorId, - applicationId(), - driverUrl, - conf.getExecutorEnv, - driverPod, - nodeToLocalTaskCount) - try { - (executorId, kubernetesClient.pods.create(executorPod)) - } catch { - case throwable: Throwable => - logError("Failed to allocate executor pod.", throwable) - throw throwable - } - } - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { totalExpectedExecutors.set(requestedTotal) true } override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { + val podsToDelete = mutable.Buffer[Pod]() RUNNING_EXECUTOR_PODS_LOCK.synchronized { for (executor <- executorIds) { val maybeRemovedExecutor = runningExecutorsToPods.remove(executor) maybeRemovedExecutor.foreach { executorPod => - kubernetesClient.pods().delete(executorPod) disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod) runningPodsToExecutors.remove(executorPod.getMetadata.getName) + podsToDelete += executorPod } if (maybeRemovedExecutor.isEmpty) { logWarning(s"Unable to remove pod for unknown executor $executor") } } } + kubernetesClient.pods().delete(podsToDelete: _*) true } def getExecutorPodByIP(podIP: String): Option[Pod] = { - // Note: Per https://github.com/databricks/scala-style-guide#concurrency, we don't - // want to be switching to scala.collection.concurrent.Map on - // executorPodsByIPs. val pod = executorPodsByIPs.get(podIP) Option(pod) } From 4b3213422e6e67b11de7b627ad46d4031043be0e Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 1 Nov 2017 16:25:43 -0700 Subject: [PATCH 14/29] Various fixes to the scheduler - Move Kubernetes client calls out of synchronized blocks to prevent locking with HTTP connection lag - Fix a bug where pods that fail to launch through the APi are not retried - Remove the map from executor pod name to executor ID by using the Pod's labels to get the same information without having to track extra state. --- .../cluster/k8s/ExecutorPodFactory.scala | 4 +- .../KubernetesClusterSchedulerBackend.scala | 85 +++++++++-------- .../cluster/k8s/ExecutorPodFactorySuite.scala | 18 ++-- ...bernetesClusterSchedulerBackendSuite.scala | 95 +++++++++++++------ 4 files changed, 124 insertions(+), 78 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 03e8e9d255349..8502081ae32f9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -138,14 +138,14 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() } }.getOrElse(Seq.empty[EnvVar]) - val executorEnv = Seq( + val executorEnv = (Seq( (ENV_EXECUTOR_PORT, executorPort.toString), (ENV_DRIVER_URL, driverUrl), // Executor backend expects integral value for executor cores, so round it up to an int. (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString), (ENV_EXECUTOR_MEMORY, executorMemoryString), (ENV_APPLICATION_ID, applicationId), - (ENV_EXECUTOR_ID, executorId)) + (ENV_EXECUTOR_ID, executorId)) ++ executorEnvs) .map(env => new EnvVarBuilder() .withName(env._1) .withValue(env._2) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index a567a7a67da08..6656235361395 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -50,12 +50,8 @@ private[spark] class KubernetesClusterSchedulerBackend( private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) private val RUNNING_EXECUTOR_PODS_LOCK = new Object - // Indexed by executor IDs @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK") private val runningExecutorsToPods = new mutable.HashMap[String, Pod] - // Indexed by executor pod names - @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK") - private val runningPodsToExecutors = new mutable.HashMap[String, String] private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]() private val podsWithKnownExitReasons = new ConcurrentHashMap[String, ExecutorExited]() private val disconnectedPodsByExecutorIdPendingRemoval = new ConcurrentHashMap[String, Pod]() @@ -117,7 +113,6 @@ private[spark] class KubernetesClusterSchedulerBackend( } else if (currentTotalExpectedExecutors <= runningExecutorsToPods.size) { logDebug("Maximum allowed executor limit reached. Not scaling up further.") } else { - val nodeToLocalTaskCount = getNodesWithLocalTaskCounts for (i <- 0 until math.min( currentTotalExpectedExecutors - runningExecutorsToPods.size, podAllocationSize)) { val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString @@ -127,7 +122,16 @@ private[spark] class KubernetesClusterSchedulerBackend( driverUrl, conf.getExecutorEnv, driverPod, - nodeToLocalTaskCount) + currentNodeToLocalTaskCount) + require(executorPod.getMetadata.getLabels.containsKey(SPARK_EXECUTOR_ID_LABEL), + s"Illegal internal state for pod with name ${executorPod.getMetadata.getName} - all" + + s" executor pods must contain the label $SPARK_EXECUTOR_ID_LABEL.") + val resolvedExecutorIdLabel = executorPod.getMetadata.getLabels.get( + SPARK_EXECUTOR_ID_LABEL) + require(resolvedExecutorIdLabel == executorId, + s"Illegal internal state for pod with name ${executorPod.getMetadata.getName} - all" + + s" executor pods must map the label with key ${SPARK_EXECUTOR_ID_LABEL} to the" + + s" executor's ID. This label mapped instead to: $resolvedExecutorIdLabel.") executorsToAllocate(executorId) = executorPod logInfo( s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}") @@ -143,8 +147,6 @@ private[spark] class KubernetesClusterSchedulerBackend( case (executorId, attemptedAllocatedExecutor) => attemptedAllocatedExecutor.map { successfullyAllocatedExecutor => runningExecutorsToPods.put(executorId, successfullyAllocatedExecutor) - runningPodsToExecutors.put( - successfullyAllocatedExecutor.getMetadata.getName, executorId) } } } @@ -166,11 +168,12 @@ private[spark] class KubernetesClusterSchedulerBackend( // We keep around executors that have exit conditions caused by the application. This // allows them to be debugged later on. Otherwise, mark them as to be deleted from the // the API server. - if (!executorExited.exitCausedByApp) { + if (executorExited.exitCausedByApp) { + logInfo(s"Executor $executorId exited because of the application.") + deleteExecutorFromDataStructures(executorId) + } else { logInfo(s"Executor $executorId failed because of a framework error.") deleteExecutorFromClusterAndDataStructures(executorId) - } else { - logInfo(s"Executor $executorId exited because of the application.") } } } @@ -187,19 +190,20 @@ private[spark] class KubernetesClusterSchedulerBackend( } def deleteExecutorFromClusterAndDataStructures(executorId: String): Unit = { + deleteExecutorFromDataStructures(executorId) + .foreach(pod => kubernetesClient.pods().delete(pod)) + } + + def deleteExecutorFromDataStructures(executorId: String): Option[Pod] = { disconnectedPodsByExecutorIdPendingRemoval.remove(executorId) executorReasonCheckAttemptCounts -= executorId - podsWithKnownExitReasons -= executorId - val maybeExecutorPodToDelete = RUNNING_EXECUTOR_PODS_LOCK.synchronized { - runningExecutorsToPods.remove(executorId).map { pod => - runningPodsToExecutors.remove(pod.getMetadata.getName) - pod - }.orElse { + podsWithKnownExitReasons.remove(executorId) + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + runningExecutorsToPods.remove(executorId).orElse { logWarning(s"Unable to remove pod for unknown executor $executorId") None } } - maybeExecutorPodToDelete.foreach(pod => kubernetesClient.pods().delete(pod)) } } @@ -231,14 +235,10 @@ private[spark] class KubernetesClusterSchedulerBackend( super.stop() // then delete the executor pods - // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context. - // When using Utils.tryLogNonFatalError some of the code fails but without any logs or - // indication as to why. Utils.tryLogNonFatalError { val executorPodsToDelete = RUNNING_EXECUTOR_PODS_LOCK.synchronized { val runningExecutorPodsCopy = Seq(runningExecutorsToPods.values.toSeq: _*) runningExecutorsToPods.clear() - runningPodsToExecutors.clear() runningExecutorPodsCopy } kubernetesClient.pods().delete(executorPodsToDelete: _*) @@ -288,7 +288,6 @@ private[spark] class KubernetesClusterSchedulerBackend( val maybeRemovedExecutor = runningExecutorsToPods.remove(executor) maybeRemovedExecutor.foreach { executorPod => disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod) - runningPodsToExecutors.remove(executorPod.getMetadata.getName) podsToDelete += executorPod } if (maybeRemovedExecutor.isEmpty) { @@ -300,11 +299,6 @@ private[spark] class KubernetesClusterSchedulerBackend( true } - def getExecutorPodByIP(podIP: String): Option[Pod] = { - val pod = executorPodsByIPs.get(podIP) - Option(pod) - } - private class ExecutorPodsWatcher extends Watcher[Pod] { private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1 @@ -316,21 +310,33 @@ private[spark] class KubernetesClusterSchedulerBackend( val clusterNodeName = pod.getSpec.getNodeName logInfo(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") executorPodsByIPs.put(podIP, pod) - } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) || - action == Action.DELETED || action == Action.ERROR) { + } else if (action == Action.DELETED || action == Action.ERROR) { + val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL) + require(executorId != null, "Unexpected pod metadata; expected all executor pods" + + s" to have label $SPARK_EXECUTOR_ID_LABEL.") val podName = pod.getMetadata.getName val podIP = pod.getStatus.getPodIP logDebug(s"Executor pod $podName at IP $podIP was at $action.") if (podIP != null) { executorPodsByIPs.remove(podIP) } - if (action == Action.ERROR) { + val executorExitReason = if (action == Action.ERROR) { logWarning(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) - handleErroredPod(pod) + executorExitReasonOnError(pod) } else if (action == Action.DELETED) { logWarning(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason) - handleDeletedPod(pod) + executorExitReasonOnDelete(pod) + } else { + throw new IllegalStateException( + s"Unknown action that should only be DELETED or ERROR: $action") + } + podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason) + if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) { + log.warn(s"Executor with id $executorId was not marked as disconnected, but the" + + s" watch received an event of type $action for this executor. The executor may" + + s" have failed to start in the first place and never registered with the driver.") } + disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) } } @@ -356,15 +362,16 @@ private[spark] class KubernetesClusterSchedulerBackend( } def isPodAlreadyReleased(pod: Pod): Boolean = { + val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL) RUNNING_EXECUTOR_PODS_LOCK.synchronized { - !runningPodsToExecutors.contains(pod.getMetadata.getName) + !runningExecutorsToPods.contains(executorId) } } - def handleErroredPod(pod: Pod): Unit = { + def executorExitReasonOnError(pod: Pod): ExecutorExited = { val containerExitStatus = getExecutorExitStatus(pod) // container was probably actively killed by the driver. - val exitReason = if (isPodAlreadyReleased(pod)) { + if (isPodAlreadyReleased(pod)) { ExecutorExited(containerExitStatus, exitCausedByApp = false, s"Container in pod ${pod.getMetadata.getName} exited from explicit termination" + " request.") @@ -373,18 +380,16 @@ private[spark] class KubernetesClusterSchedulerBackend( s"exited with exit status code $containerExitStatus." ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason) } - podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason) } - def handleDeletedPod(pod: Pod): Unit = { + def executorExitReasonOnDelete(pod: Pod): ExecutorExited = { val exitMessage = if (isPodAlreadyReleased(pod)) { s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request." } else { s"Pod ${pod.getMetadata.getName} deleted or lost." } - val exitReason = ExecutorExited( + ExecutorExited( getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage) - podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index 03756a739c47e..d1e64375a8b84 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -25,7 +25,7 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.config._ -import org.apache.spark.deploy.k8s.constants +import org.apache.spark.deploy.k8s.constants._ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { private val driverPodName: String = "driver-pod" @@ -64,6 +64,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef // The executor pod name and default labels. assert(executor.getMetadata.getName === s"$executorPrefix-exec-1") assert(executor.getMetadata.getLabels.size() === 3) + assert(executor.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL) === "1") // There is exactly 1 container with no volume mounts and default memory limits. // Default memory limit is 1024M + 384M (minimum overhead constant). @@ -120,14 +121,13 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef // Check that the expected environment variables are present. private def checkEnv(executor: Pod, additionalEnvVars: Map[String, String]): Unit = { val defaultEnvs = Map( - constants.ENV_EXECUTOR_ID -> "1", - constants.ENV_DRIVER_URL -> "dummy", - constants.ENV_EXECUTOR_CORES -> "1", - constants.ENV_EXECUTOR_MEMORY -> "1g", - constants.ENV_APPLICATION_ID -> "dummy", - constants.ENV_MOUNTED_CLASSPATH -> "/var/spark-data/spark-jars/*", - constants.ENV_EXECUTOR_POD_IP -> null, - constants.ENV_EXECUTOR_PORT -> "10000") ++ additionalEnvVars + ENV_EXECUTOR_ID -> "1", + ENV_DRIVER_URL -> "dummy", + ENV_EXECUTOR_CORES -> "1", + ENV_EXECUTOR_MEMORY -> "1g", + ENV_APPLICATION_ID -> "dummy", + ENV_EXECUTOR_POD_IP -> null, + ENV_EXECUTOR_PORT -> "10000") ++ additionalEnvVars assert(executor.getSpec.getContainers.size() === 1) assert(executor.getSpec.getContainers.get(0).getEnv().size() === defaultEnvs.size) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 3ce9478fde74b..1ca169217fc3f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -198,12 +198,12 @@ class KubernetesClusterSchedulerBackendSuite val scheduler = newSchedulerBackend() scheduler.start() requestExecutorRunnable.getValue.run() - expectPodCreationWithId(1, FIRST_EXECUTOR_POD) - expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + val secondResolvedPod = expectPodCreationWithId(2, SECOND_EXECUTOR_POD) when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) allocatorRunnable.getValue.run() - verify(podOperations).create(FIRST_EXECUTOR_POD) - verify(podOperations).create(SECOND_EXECUTOR_POD) + verify(podOperations).create(firstResolvedPod) + verify(podOperations).create(secondResolvedPod) } test("Killing executors deletes the executor pods") { @@ -213,15 +213,15 @@ class KubernetesClusterSchedulerBackendSuite val scheduler = newSchedulerBackend() scheduler.start() requestExecutorRunnable.getValue.run() - expectPodCreationWithId(1, FIRST_EXECUTOR_POD) - expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + val secondResolvedPod = expectPodCreationWithId(2, SECOND_EXECUTOR_POD) when(podOperations.create(any(classOf[Pod]))) .thenAnswer(AdditionalAnswers.returnsFirstArg()) allocatorRunnable.getValue.run() scheduler.doKillExecutors(Seq("2")) requestExecutorRunnable.getAllValues.asScala.last.run() - verify(podOperations).delete(SECOND_EXECUTOR_POD) - verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + verify(podOperations).delete(secondResolvedPod) + verify(podOperations, never()).delete(firstResolvedPod) } test("Executors should be requested in batches.") { @@ -233,18 +233,18 @@ class KubernetesClusterSchedulerBackendSuite requestExecutorRunnable.getValue.run() when(podOperations.create(any(classOf[Pod]))) .thenAnswer(AdditionalAnswers.returnsFirstArg()) - expectPodCreationWithId(1, FIRST_EXECUTOR_POD) - expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + val secondResolvedPod = expectPodCreationWithId(2, SECOND_EXECUTOR_POD) allocatorRunnable.getValue.run() - verify(podOperations).create(FIRST_EXECUTOR_POD) - verify(podOperations, never()).create(SECOND_EXECUTOR_POD) + verify(podOperations).create(firstResolvedPod) + verify(podOperations, never()).create(secondResolvedPod) val registerFirstExecutorMessage = RegisterExecutor( "1", mock[RpcEndpointRef], "localhost", 1, Map.empty[String, String]) when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) .apply(registerFirstExecutorMessage) allocatorRunnable.getValue.run() - verify(podOperations).create(SECOND_EXECUTOR_POD) + verify(podOperations).create(secondResolvedPod) } test("Scaled down executors should be cleaned up") { @@ -258,7 +258,7 @@ class KubernetesClusterSchedulerBackendSuite requestExecutorRunnable.getValue.run() when(podOperations.create(any(classOf[Pod]))) .thenAnswer(AdditionalAnswers.returnsFirstArg()) - expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + val resolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) allocatorRunnable.getValue.run() val executorEndpointRef = mock[RpcEndpointRef] when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) @@ -273,16 +273,16 @@ class KubernetesClusterSchedulerBackendSuite requestExecutorRunnable.getAllValues.asScala.last.run() scheduler.doKillExecutors(Seq("1")) requestExecutorRunnable.getAllValues.asScala.last.run() - verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) + verify(podOperations, times(1)).delete(resolvedPod) driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) - val exitedPod = exitPod(FIRST_EXECUTOR_POD, 0) + val exitedPod = exitPod(resolvedPod, 0) executorPodsWatcherArgument.getValue.eventReceived(Action.DELETED, exitedPod) allocatorRunnable.getValue.run() // No more deletion attempts of the executors. // This is graceful termination and should not be detected as a failure. - verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) + verify(podOperations, times(1)).delete(resolvedPod) verify(driverEndpointRef, times(1)).ask[Boolean]( RemoveExecutor("1", ExecutorExited( 0, @@ -298,7 +298,7 @@ class KubernetesClusterSchedulerBackendSuite val scheduler = newSchedulerBackend() scheduler.start() - expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) requestExecutorRunnable.getValue.run() allocatorRunnable.getValue.run() @@ -311,20 +311,20 @@ class KubernetesClusterSchedulerBackendSuite .apply(registerFirstExecutorMessage) driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) executorPodsWatcherArgument.getValue.eventReceived( - Action.ERROR, exitPod(FIRST_EXECUTOR_POD, 1)) + Action.ERROR, exitPod(firstResolvedPod, 1)) // A replacement executor should be created but the error pod should persist. - expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + val replacementPod = expectPodCreationWithId(2, SECOND_EXECUTOR_POD) scheduler.doRequestTotalExecutors(1) requestExecutorRunnable.getValue.run() allocatorRunnable.getAllValues.asScala.last.run() + verify(podOperations, never()).delete(firstResolvedPod) verify(driverEndpointRef).ask[Boolean]( RemoveExecutor("1", ExecutorExited( 1, exitCausedByApp = true, s"Pod ${FIRST_EXECUTOR_POD.getMetadata.getName}'s executor container exited with" + " exit status code 1."))) - verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) } test("Executors disconnected due to unknown reasons are deleted and replaced.") { @@ -334,7 +334,7 @@ class KubernetesClusterSchedulerBackendSuite val scheduler = newSchedulerBackend() scheduler.start() - expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) requestExecutorRunnable.getValue.run() allocatorRunnable.getValue.run() @@ -352,13 +352,48 @@ class KubernetesClusterSchedulerBackendSuite verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) } - expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + val recreatedResolvedPod = expectPodCreationWithId(2, SECOND_EXECUTOR_POD) allocatorRunnable.getValue.run() - verify(podOperations).delete(FIRST_EXECUTOR_POD) + verify(podOperations).delete(firstResolvedPod) verify(driverEndpointRef).ask[Boolean]( RemoveExecutor("1", SlaveLost("Executor lost for unknown reasons."))) } + test("Executors that fail to start on the Kubernetes API call rebuild in the next batch.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend() + scheduler.start() + val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + when(podOperations.create(firstResolvedPod)) + .thenThrow(new RuntimeException("test")) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getValue.run() + verify(podOperations, times(1)).create(firstResolvedPod) + val recreatedResolvedPod = expectPodCreationWithId(2, FIRST_EXECUTOR_POD) + allocatorRunnable.getValue.run() + verify(podOperations).create(recreatedResolvedPod) + } + + test("Executors that are initially created but the watch notices them fail are rebuilt" + + " in the next batch.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend() + scheduler.start() + val firstResolvedPod = expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + when(podOperations.create(FIRST_EXECUTOR_POD)).thenAnswer(AdditionalAnswers.returnsFirstArg()) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getValue.run() + verify(podOperations, times(1)).create(firstResolvedPod) + executorPodsWatcherArgument.getValue.eventReceived(Action.ERROR, firstResolvedPod) + val recreatedResolvedPod = expectPodCreationWithId(2, FIRST_EXECUTOR_POD) + allocatorRunnable.getValue.run() + verify(podOperations).create(recreatedResolvedPod) + } + private def newSchedulerBackend(): KubernetesClusterSchedulerBackend = { new KubernetesClusterSchedulerBackend( taskSchedulerImpl, @@ -373,7 +408,7 @@ class KubernetesClusterSchedulerBackendSuite } private def exitPod(basePod: Pod, exitCode: Int): Pod = { - new PodBuilder(FIRST_EXECUTOR_POD) + new PodBuilder(basePod) .editStatus() .addNewContainerStatus() .withNewState() @@ -386,13 +421,19 @@ class KubernetesClusterSchedulerBackendSuite .build() } - private def expectPodCreationWithId(executorId: Int, expectedPod: Pod): Unit = { + private def expectPodCreationWithId(executorId: Int, expectedPod: Pod): Pod = { + val resolvedPod = new PodBuilder(expectedPod) + .editMetadata() + .addToLabels(SPARK_EXECUTOR_ID_LABEL, executorId.toString) + .endMetadata() + .build() when(executorPodFactory.createExecutorPod( executorId.toString, APP_ID, DRIVER_URL, sparkConf.getExecutorEnv, driverPod, - Map.empty)).thenReturn(expectedPod) + Map.empty)).thenReturn(resolvedPod) + resolvedPod } } From 6cf4ed7eec3f8a1798d260622ab5641db92ab13d Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 3 Nov 2017 17:19:21 -0700 Subject: [PATCH 15/29] Address comments --- .../org/apache/spark/deploy/k8s/ConfigurationUtils.scala | 9 --------- .../spark/scheduler/cluster/k8s/ExecutorPodFactory.scala | 7 +++++-- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 4 ++-- .../apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala | 2 -- 4 files changed, 7 insertions(+), 15 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index 6d443d1774107..c0c6514658c15 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -34,15 +34,6 @@ private[spark] object ConfigurationUtils { fromPrefix.toMap } - def requireBothOrNeitherDefined( - opt1: Option[_], - opt2: Option[_], - errMessageWhenFirstIsMissing: String, - errMessageWhenSecondIsMissing: String): Unit = { - requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) - requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) - } - def requireSecondIfFirstIsDefined( opt1: Option[_], opt2: Option[_], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 8502081ae32f9..46da9a9aafc79 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -45,8 +45,8 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) import ExecutorPodFactoryImpl._ - private val executorExtraClasspath = sparkConf.get( - org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) + private val executorExtraClasspath = + sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, @@ -59,6 +59,9 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL), s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + s" Spark.") + require( + !executorLabels.contains(SPARK_ROLE_LABEL), + s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.") private val executorAnnotations = ConfigurationUtils.parsePrefixedKeyValuePairs( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 6656235361395..2fbb737893ff8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -87,12 +87,12 @@ private[spark] class KubernetesClusterSchedulerBackend( private val initialExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(conf) private val podAllocationInterval = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) - require(podAllocationInterval > 0, s"Allocation batch delay " + + require(podAllocationInterval > 0, "Allocation batch delay " + s"${KUBERNETES_ALLOCATION_BATCH_DELAY} " + s"is ${podAllocationInterval}, should be a positive integer") private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) - require(podAllocationSize > 0, s"Allocation batch size " + + require(podAllocationSize > 0, "Allocation batch size " + s"${KUBERNETES_ALLOCATION_BATCH_SIZE} " + s"is ${podAllocationSize}, should be a positive integer") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index ca6f62c517604..9c1472cb50e3a 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -133,8 +133,6 @@ object YarnSparkHadoopUtil { val ANY_HOST = "*" - val DEFAULT_NUMBER_EXECUTORS = 2 - // All RM requests are issued with same priority : we do not (yet) have any distinction between // request types (like map/reduce in hadoop for example) val RM_REQUEST_PRIORITY = Priority.newInstance(1) From 1f271bee5b43c03fab6d2c55d96a93f417cc6915 Mon Sep 17 00:00:00 2001 From: foxish Date: Mon, 13 Nov 2017 07:26:31 -0800 Subject: [PATCH 16/29] Update fabric8 client version to 3.0.0 --- resource-managers/kubernetes/core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 1637c0f7aa716..c5bdd1195e6fc 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -29,7 +29,7 @@ Spark Project Kubernetes kubernetes - 2.2.13 + 3.0.0 From 71a971f2108fd9e04532fe73ecbd1ec00b36d132 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 13 Nov 2017 10:45:32 -0800 Subject: [PATCH 17/29] Addressed more comments --- .../spark/deploy/k8s/ConfigurationUtils.scala | 19 ++--- .../k8s/SparkKubernetesClientFactory.scala | 4 +- .../org/apache/spark/deploy/k8s/config.scala | 3 +- .../cluster/k8s/ExecutorPodFactory.scala | 9 +-- .../k8s/KubernetesClusterManager.scala | 6 +- .../KubernetesClusterSchedulerBackend.scala | 80 ++++++++++--------- 6 files changed, 62 insertions(+), 59 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index c0c6514658c15..63b28302b1baa 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -20,6 +20,16 @@ package org.apache.spark.deploy.k8s import org.apache.spark.SparkConf private[spark] object ConfigurationUtils { + + /** + * Extract and parse Spark configuration properties with a given name prefix and + * return the result as a Map. Keys must not have more than one value. + * + * @param sparkConf Spark configuration + * @param prefix the given property name prefix + * @param configType a descriptive note on the type of entities of interest + * @return a Map storing the configuration property keys and values + */ def parsePrefixedKeyValuePairs( sparkConf: SparkConf, prefix: String, @@ -34,15 +44,6 @@ private[spark] object ConfigurationUtils { fromPrefix.toMap } - def requireSecondIfFirstIsDefined( - opt1: Option[_], - opt2: Option[_], - errMessageWhenSecondIsMissing: String): Unit = { - opt1.foreach { _ => - require(opt2.isDefined, errMessageWhenSecondIsMissing) - } - } - def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { opt1.foreach { _ => require(opt2.isEmpty, errMessage) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 444144b69c341..ab2d118ca72a1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -51,8 +51,8 @@ private[spark] object SparkKubernetesClientFactory { ConfigurationUtils.requireNandDefined( oauthTokenFile, oauthTokenValue, - s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" + - s" value $oauthTokenConf.") + s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a " + + s"value $oauthTokenConf.") val caCertFile = sparkConf .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala index f49056e06554c..74cd0c45a98cd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala @@ -44,7 +44,6 @@ package object config extends Logging { .stringConf .createWithDefault("IfNotPresent") - private[spark] val APISERVER_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver" private[spark] val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX = @@ -95,12 +94,14 @@ package object config extends Logging { ConfigBuilder("spark.kubernetes.allocation.batch.size") .doc("Number of pods to launch at once in each round of executor allocation.") .intConf + .checkValue(value => value > 0, "Allocation batch size should be a positive integer") .createWithDefault(5) private[spark] val KUBERNETES_ALLOCATION_BATCH_DELAY = ConfigBuilder("spark.kubernetes.allocation.batch.delay") .doc("Number of seconds to wait between each round of executor allocation.") .longConf + .checkValue(value => value > 0, s"Allocation batch delay should be a positive integer") .createWithDefault(1) private[spark] val KUBERNETES_EXECUTOR_LIMIT_CORES = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 46da9a9aafc79..87c289e1f8793 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -20,7 +20,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ @@ -77,11 +77,8 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE) private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) - private val blockmanagerPort = sparkConf + private val blockManagerPort = sparkConf .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) - private val kubernetesDriverPodName = sparkConf - .get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(throw new SparkException("Must specify the driver pod name")) private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) @@ -163,7 +160,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq val requiredPorts = Seq( (EXECUTOR_PORT_NAME, executorPort), - (BLOCK_MANAGER_PORT_NAME, blockmanagerPort)) + (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) .map(port => { new ContainerPortBuilder() .withName(port._1) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 74aac6d211f40..5e4bd7c41594e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -36,8 +36,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit new TaskSchedulerImpl(sc) } - override def createSchedulerBackend(sc: SparkContext, masterURL: String, scheduler: TaskScheduler) - : SchedulerBackend = { + override def createSchedulerBackend( + sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = { val sparkConf = sc.getConf val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 2fbb737893ff8..3ceb15fc1aab7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -87,14 +87,8 @@ private[spark] class KubernetesClusterSchedulerBackend( private val initialExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(conf) private val podAllocationInterval = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) - require(podAllocationInterval > 0, "Allocation batch delay " + - s"${KUBERNETES_ALLOCATION_BATCH_DELAY} " + - s"is ${podAllocationInterval}, should be a positive integer") private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) - require(podAllocationSize > 0, "Allocation batch size " + - s"${KUBERNETES_ALLOCATION_BATCH_SIZE} " + - s"is ${podAllocationSize}, should be a positive integer") private val allocatorRunnable = new Runnable { @@ -304,39 +298,40 @@ private[spark] class KubernetesClusterSchedulerBackend( private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1 override def eventReceived(action: Action, pod: Pod): Unit = { - if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running" - && pod.getMetadata.getDeletionTimestamp == null) { - val podIP = pod.getStatus.getPodIP - val clusterNodeName = pod.getSpec.getNodeName - logInfo(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") - executorPodsByIPs.put(podIP, pod) - } else if (action == Action.DELETED || action == Action.ERROR) { - val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL) - require(executorId != null, "Unexpected pod metadata; expected all executor pods" + - s" to have label $SPARK_EXECUTOR_ID_LABEL.") - val podName = pod.getMetadata.getName - val podIP = pod.getStatus.getPodIP - logDebug(s"Executor pod $podName at IP $podIP was at $action.") - if (podIP != null) { - executorPodsByIPs.remove(podIP) - } - val executorExitReason = if (action == Action.ERROR) { - logWarning(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) - executorExitReasonOnError(pod) - } else if (action == Action.DELETED) { - logWarning(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason) - executorExitReasonOnDelete(pod) - } else { - throw new IllegalStateException( - s"Unknown action that should only be DELETED or ERROR: $action") - } - podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason) - if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) { - log.warn(s"Executor with id $executorId was not marked as disconnected, but the" + - s" watch received an event of type $action for this executor. The executor may" + - s" have failed to start in the first place and never registered with the driver.") - } - disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) + action match { + case Action.MODIFIED if (pod.getStatus.getPhase == "Running" + && pod.getMetadata.getDeletionTimestamp == null) => + val podIP = pod.getStatus.getPodIP + val clusterNodeName = pod.getSpec.getNodeName + logInfo(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") + executorPodsByIPs.put(podIP, pod) + case Action.DELETED | Action.ERROR => + val executorId = getExecutorId(pod) + val podName = pod.getMetadata.getName + val podIP = pod.getStatus.getPodIP + logDebug(s"Executor pod $podName at IP $podIP was at $action.") + if (podIP != null) { + executorPodsByIPs.remove(podIP) + } + + val executorExitReason = if (action == Action.ERROR) { + logWarning(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) + executorExitReasonOnError(pod) + } else if (action == Action.DELETED) { + logWarning(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason) + executorExitReasonOnDelete(pod) + } else { + throw new IllegalStateException( + s"Unknown action that should only be DELETED or ERROR: $action") + } + podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason) + + if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) { + log.warn(s"Executor with id $executorId was not marked as disconnected, but the" + + s" watch received an event of type $action for this executor. The executor may" + + s" have failed to start in the first place and never registered with the driver.") + } + disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) } } @@ -391,6 +386,13 @@ private[spark] class KubernetesClusterSchedulerBackend( ExecutorExited( getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage) } + + def getExecutorId(pod: Pod): String = { + val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL) + require(executorId != null, "Unexpected pod metadata; expected all executor pods " + + s"to have label $SPARK_EXECUTOR_ID_LABEL.") + executorId + } } override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { From 0ab9ca7b2056c0de7a820fa7bb9391227bcf5275 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 13 Nov 2017 23:06:35 -0800 Subject: [PATCH 18/29] One more round of comments --- resource-managers/kubernetes/core/pom.xml | 10 +- .../spark/deploy/k8s/ConfigurationUtils.scala | 9 +- .../k8s/SparkKubernetesClientFactory.scala | 8 +- .../org/apache/spark/deploy/k8s/config.scala | 46 ++++----- .../cluster/k8s/ExecutorPodFactory.scala | 34 +++---- .../KubernetesClusterSchedulerBackend.scala | 94 +++++++++---------- 6 files changed, 94 insertions(+), 107 deletions(-) diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index c5bdd1195e6fc..204fdc7fa4afb 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -54,15 +54,7 @@ com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.core - jackson-annotations + * com.fasterxml.jackson.dataformat diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index 63b28302b1baa..56444c16fd0ee 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -34,14 +34,7 @@ private[spark] object ConfigurationUtils { sparkConf: SparkConf, prefix: String, configType: String): Map[String, String] = { - val fromPrefix = sparkConf.getAllWithPrefix(prefix) - fromPrefix.groupBy(_._1).foreach { - case (key, values) => - require(values.size == 1, - s"Cannot have multiple values for a given $configType key, got key $key with" + - s" values $values") - } - fromPrefix.toMap + sparkConf.getAllWithPrefix(prefix).toMap } def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index ab2d118ca72a1..38cda4565f45b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -40,13 +40,13 @@ private[spark] object SparkKubernetesClientFactory { namespace: Option[String], kubernetesAuthConfPrefix: String, sparkConf: SparkConf, - maybeServiceAccountToken: Option[File], - maybeServiceAccountCaCert: Option[File]): KubernetesClient = { + defaultServiceAccountToken: Option[File], + defaultServiceAccountCaCert: Option[File]): KubernetesClient = { val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX" val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX" val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf) .map(new File(_)) - .orElse(maybeServiceAccountToken) + .orElse(defaultServiceAccountToken) val oauthTokenValue = sparkConf.getOption(oauthTokenConf) ConfigurationUtils.requireNandDefined( oauthTokenFile, @@ -56,7 +56,7 @@ private[spark] object SparkKubernetesClientFactory { val caCertFile = sparkConf .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX") - .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath)) + .orElse(defaultServiceAccountCaCert.map(_.getAbsolutePath)) val clientKeyFile = sparkConf .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") val clientCertFile = sparkConf diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala index 74cd0c45a98cd..c6eff28d3b87b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala @@ -16,14 +16,14 @@ */ package org.apache.spark.deploy.k8s -import org.apache.spark.{SPARK_VERSION => sparkVersion} +import org.apache.spark.SPARK_VERSION import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit -package object config extends Logging { +private[spark] object config extends Logging { - private[spark] val KUBERNETES_NAMESPACE = + val KUBERNETES_NAMESPACE = ConfigBuilder("spark.kubernetes.namespace") .doc("The namespace that will be used for running the driver and executor pods. When using" + " spark-submit in cluster mode, this can also be passed to spark-submit via the" + @@ -31,30 +31,30 @@ package object config extends Logging { .stringConf .createWithDefault("default") - private[spark] val EXECUTOR_DOCKER_IMAGE = + val EXECUTOR_DOCKER_IMAGE = ConfigBuilder("spark.kubernetes.executor.docker.image") .doc("Docker image to use for the executors. Specify this using the standard Docker tag" + " format.") .stringConf - .createWithDefault(s"spark-executor:$sparkVersion") + .createWithDefault(s"spark-executor:$SPARK_VERSION") - private[spark] val DOCKER_IMAGE_PULL_POLICY = + val DOCKER_IMAGE_PULL_POLICY = ConfigBuilder("spark.kubernetes.docker.image.pullPolicy") .doc("Docker image pull policy when pulling any docker image in Kubernetes integration") .stringConf .createWithDefault("IfNotPresent") - private[spark] val APISERVER_AUTH_DRIVER_CONF_PREFIX = + val APISERVER_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver" - private[spark] val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX = + val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX = "spark.kubernetes.authenticate.driver.mounted" - private[spark] val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken" - private[spark] val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile" - private[spark] val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile" - private[spark] val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile" - private[spark] val CA_CERT_FILE_CONF_SUFFIX = "caCertFile" + val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken" + val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile" + val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile" + val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile" + val CA_CERT_FILE_CONF_SUFFIX = "caCertFile" - private[spark] val KUBERNETES_SERVICE_ACCOUNT_NAME = + val KUBERNETES_SERVICE_ACCOUNT_NAME = ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName") .doc("Service account that is used when running the driver pod. The driver pod uses" + " this service account when requesting executor pods from the API server. If specific" + @@ -66,7 +66,7 @@ package object config extends Logging { // Note that while we set a default for this when we start up the // scheduler, the specific default value is dynamically determined // based on the executor memory. - private[spark] val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD = + val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.kubernetes.executor.memoryOverhead") .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" + " is memory that accounts for things like VM overheads, interned strings, other native" + @@ -74,41 +74,41 @@ package object config extends Logging { .bytesConf(ByteUnit.MiB) .createOptional - private[spark] val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." - private[spark] val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." + val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." + val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." - private[spark] val KUBERNETES_DRIVER_POD_NAME = + val KUBERNETES_DRIVER_POD_NAME = ConfigBuilder("spark.kubernetes.driver.pod.name") .doc("Name of the driver pod.") .stringConf .createOptional - private[spark] val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = + val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = ConfigBuilder("spark.kubernetes.executor.podNamePrefix") .doc("Prefix to use in front of the executor pod names.") .internal() .stringConf .createWithDefault("spark") - private[spark] val KUBERNETES_ALLOCATION_BATCH_SIZE = + val KUBERNETES_ALLOCATION_BATCH_SIZE = ConfigBuilder("spark.kubernetes.allocation.batch.size") .doc("Number of pods to launch at once in each round of executor allocation.") .intConf .checkValue(value => value > 0, "Allocation batch size should be a positive integer") .createWithDefault(5) - private[spark] val KUBERNETES_ALLOCATION_BATCH_DELAY = + val KUBERNETES_ALLOCATION_BATCH_DELAY = ConfigBuilder("spark.kubernetes.allocation.batch.delay") .doc("Number of seconds to wait between each round of executor allocation.") .longConf .checkValue(value => value > 0, s"Allocation batch delay should be a positive integer") .createWithDefault(1) - private[spark] val KUBERNETES_EXECUTOR_LIMIT_CORES = + val KUBERNETES_EXECUTOR_LIMIT_CORES = ConfigBuilder("spark.kubernetes.executor.limit.cores") .doc("Specify the hard cpu limit for a single executor pod") .stringConf .createOptional - private[spark] val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." + val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 87c289e1f8793..764167a68c229 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -27,10 +27,13 @@ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.util.Utils /** - * Configures executor pods. Construct one of these with a SparkConf to set up properties that are - * common across all executors. Then, pass in dynamic parameters into createExecutorPod. + * A factory class for configuring and creating executor pods. */ private[spark] trait ExecutorPodFactory { + + /** + * Configure and construct an executor pod with the given parameters. + */ def createExecutorPod( executorId: String, applicationId: String, @@ -161,12 +164,12 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) val requiredPorts = Seq( (EXECUTOR_PORT_NAME, executorPort), (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) - .map(port => { + .map { case (name, port) => new ContainerPortBuilder() - .withName(port._1) - .withContainerPort(port._2) + .withName(name) + .withContainerPort(port) .build() - }) + } val executorContainer = new ContainerBuilder() .withName(s"executor") @@ -202,16 +205,15 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) .endSpec() .build() - val containerWithExecutorLimitCores = executorLimitCores.map { - limitCores => - val executorCpuLimitQuantity = new QuantityBuilder(false) - .withAmount(limitCores) - .build() - new ContainerBuilder(executorContainer) - .editResources() - .addToLimits("cpu", executorCpuLimitQuantity) - .endResources() - .build() + val containerWithExecutorLimitCores = executorLimitCores.map { limitCores => + val executorCpuLimitQuantity = new QuantityBuilder(false) + .withAmount(limitCores) + .build() + new ContainerBuilder(executorContainer) + .editResources() + .addToLimits("cpu", executorCpuLimitQuantity) + .endResources() + .build() }.getOrElse(executorContainer) new PodBuilder(executorPod) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 3ceb15fc1aab7..ccdf1e1e9f4a6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -69,7 +69,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .withName(kubernetesDriverPodName) .get() - override val minRegisteredRatio = + protected override val minRegisteredRatio = if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { 0.8 } else { @@ -77,7 +77,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } private val executorWatchResource = new AtomicReference[Closeable] - protected val totalExpectedExecutors = new AtomicInteger(0) + private val totalExpectedExecutors = new AtomicInteger(0) private val driverUrl = RpcEndpointAddress( conf.get("spark.driver.host"), @@ -98,10 +98,11 @@ private[spark] class KubernetesClusterSchedulerBackend( override def run(): Unit = { handleDisconnectedExecutors() + val executorsToAllocate = mutable.Map[String, Pod]() val currentTotalRegisteredExecutors = totalRegisteredExecutors.get val currentTotalExpectedExecutors = totalExpectedExecutors.get - val currentNodeToLocalTaskCount = getNodesWithLocalTaskCounts + val currentNodeToLocalTaskCount = getNodesWithLocalTaskCounts() if (currentTotalRegisteredExecutors < runningExecutorsToPods.size) { logDebug("Waiting for pending executors before scaling") } else if (currentTotalExpectedExecutors <= runningExecutorsToPods.size) { @@ -117,25 +118,18 @@ private[spark] class KubernetesClusterSchedulerBackend( conf.getExecutorEnv, driverPod, currentNodeToLocalTaskCount) - require(executorPod.getMetadata.getLabels.containsKey(SPARK_EXECUTOR_ID_LABEL), - s"Illegal internal state for pod with name ${executorPod.getMetadata.getName} - all" + - s" executor pods must contain the label $SPARK_EXECUTOR_ID_LABEL.") - val resolvedExecutorIdLabel = executorPod.getMetadata.getLabels.get( - SPARK_EXECUTOR_ID_LABEL) - require(resolvedExecutorIdLabel == executorId, - s"Illegal internal state for pod with name ${executorPod.getMetadata.getName} - all" + - s" executor pods must map the label with key ${SPARK_EXECUTOR_ID_LABEL} to the" + - s" executor's ID. This label mapped instead to: $resolvedExecutorIdLabel.") executorsToAllocate(executorId) = executorPod logInfo( s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}") } } + val allocatedExecutors = executorsToAllocate.mapValues { pod => Utils.tryLog { kubernetesClient.pods().create(pod) } } + RUNNING_EXECUTOR_PODS_LOCK.synchronized { allocatedExecutors.map { case (executorId, attemptedAllocatedExecutor) => @@ -184,8 +178,8 @@ private[spark] class KubernetesClusterSchedulerBackend( } def deleteExecutorFromClusterAndDataStructures(executorId: String): Unit = { - deleteExecutorFromDataStructures(executorId) - .foreach(pod => kubernetesClient.pods().delete(pod)) + deleteExecutorFromDataStructures(executorId).foreach { pod => + kubernetesClient.pods().delete(pod) } } def deleteExecutorFromDataStructures(executorId: String): Option[Pod] = { @@ -253,10 +247,10 @@ private[spark] class KubernetesClusterSchedulerBackend( * locality if an executor launches on the cluster node. */ private def getNodesWithLocalTaskCounts() : Map[String, Int] = { - val nodeToLocalTaskCount = mutable.Map[String, Int]() ++ - synchronized { - hostToLocalTaskCount - } + val nodeToLocalTaskCount = synchronized { + mutable.Map[String, Int]() ++ hostToLocalTaskCount + } + for (pod <- executorPodsByIPs.values().asScala) { // Remove cluster nodes that are running our executors already. // TODO: This prefers spreading out executors across nodes. In case users want @@ -276,19 +270,20 @@ private[spark] class KubernetesClusterSchedulerBackend( } override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { - val podsToDelete = mutable.Buffer[Pod]() - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - for (executor <- executorIds) { - val maybeRemovedExecutor = runningExecutorsToPods.remove(executor) - maybeRemovedExecutor.foreach { executorPod => - disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod) - podsToDelete += executorPod - } - if (maybeRemovedExecutor.isEmpty) { - logWarning(s"Unable to remove pod for unknown executor $executor") + val podsToDelete = RUNNING_EXECUTOR_PODS_LOCK.synchronized { + executorIds.flatMap { executorId => + runningExecutorsToPods.remove(executorId) match { + case Some(pod) => + disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) + Some(pod) + + case None => + logWarning(s"Unable to remove pod for unknown executor $executorId") + None } } } + kubernetesClient.pods().delete(podsToDelete: _*) true } @@ -298,27 +293,30 @@ private[spark] class KubernetesClusterSchedulerBackend( private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1 override def eventReceived(action: Action, pod: Pod): Unit = { + val podName = pod.getMetadata.getName + val podIP = pod.getStatus.getPodIP + action match { case Action.MODIFIED if (pod.getStatus.getPhase == "Running" - && pod.getMetadata.getDeletionTimestamp == null) => - val podIP = pod.getStatus.getPodIP + && pod.getMetadata.getDeletionTimestamp == null) => val clusterNodeName = pod.getSpec.getNodeName - logInfo(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") + logInfo(s"Executor pod $podName ready, launched at $clusterNodeName as IP $podIP.") executorPodsByIPs.put(podIP, pod) + case Action.DELETED | Action.ERROR => val executorId = getExecutorId(pod) - val podName = pod.getMetadata.getName - val podIP = pod.getStatus.getPodIP logDebug(s"Executor pod $podName at IP $podIP was at $action.") if (podIP != null) { executorPodsByIPs.remove(podIP) } val executorExitReason = if (action == Action.ERROR) { - logWarning(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) + logWarning(s"Received error event of executor pod $podName. Reason: " + + pod.getStatus.getReason) executorExitReasonOnError(pod) } else if (action == Action.DELETED) { - logWarning(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason) + logWarning(s"Received delete event of executor pod $podName. Reason: " + + pod.getStatus.getReason) executorExitReasonOnDelete(pod) } else { throw new IllegalStateException( @@ -332,6 +330,8 @@ private[spark] class KubernetesClusterSchedulerBackend( s" have failed to start in the first place and never registered with the driver.") } disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) + + case _ => logDebug(s"Received event of executor pod $podName: " + action) } } @@ -339,7 +339,7 @@ private[spark] class KubernetesClusterSchedulerBackend( logDebug("Executor pod watch closed.", cause) } - def getExecutorExitStatus(pod: Pod): Int = { + private def getExecutorExitStatus(pod: Pod): Int = { val containerStatuses = pod.getStatus.getContainerStatuses if (!containerStatuses.isEmpty) { // we assume the first container represents the pod status. This assumption may not hold @@ -349,21 +349,22 @@ private[spark] class KubernetesClusterSchedulerBackend( } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS } - def getExecutorExitStatus(containerStatus: ContainerStatus): Int = { - Option(containerStatus.getState).map(containerState => - Option(containerState.getTerminated).map(containerStateTerminated => - containerStateTerminated.getExitCode.intValue()).getOrElse(UNKNOWN_EXIT_CODE) - ).getOrElse(UNKNOWN_EXIT_CODE) + private def getExecutorExitStatus(containerStatus: ContainerStatus): Int = { + Option(containerStatus.getState).map { containerState => + Option(containerState.getTerminated).map {containerStateTerminated => + containerStateTerminated.getExitCode.intValue() + }.getOrElse(UNKNOWN_EXIT_CODE) + }.getOrElse(UNKNOWN_EXIT_CODE) } - def isPodAlreadyReleased(pod: Pod): Boolean = { + private def isPodAlreadyReleased(pod: Pod): Boolean = { val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL) RUNNING_EXECUTOR_PODS_LOCK.synchronized { !runningExecutorsToPods.contains(executorId) } } - def executorExitReasonOnError(pod: Pod): ExecutorExited = { + private def executorExitReasonOnError(pod: Pod): ExecutorExited = { val containerExitStatus = getExecutorExitStatus(pod) // container was probably actively killed by the driver. if (isPodAlreadyReleased(pod)) { @@ -377,17 +378,16 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - def executorExitReasonOnDelete(pod: Pod): ExecutorExited = { + private def executorExitReasonOnDelete(pod: Pod): ExecutorExited = { val exitMessage = if (isPodAlreadyReleased(pod)) { s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request." } else { s"Pod ${pod.getMetadata.getName} deleted or lost." } - ExecutorExited( - getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage) + ExecutorExited(getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage) } - def getExecutorId(pod: Pod): String = { + private def getExecutorId(pod: Pod): String = { val executorId = pod.getMetadata.getLabels.get(SPARK_EXECUTOR_ID_LABEL) require(executorId != null, "Unexpected pod metadata; expected all executor pods " + s"to have label $SPARK_EXECUTOR_ID_LABEL.") From 7f14b71c0254553893ac70642d0f0b77bb7687a2 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Wed, 15 Nov 2017 09:43:48 -0800 Subject: [PATCH 19/29] Added a comment regarding how failed executor pods are handled --- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index ccdf1e1e9f4a6..43249dcf42135 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -153,9 +153,12 @@ private[spark] class KubernetesClusterSchedulerBackend( } { executorExited => logWarning(s"Removing executor $executorId with loss reason " + executorExited.message) removeExecutor(executorId, executorExited) - // We keep around executors that have exit conditions caused by the application. This - // allows them to be debugged later on. Otherwise, mark them as to be deleted from the - // the API server. + // We don't delete the pod running the executor that has an exit condition caused by + // the application from the Kubernetes API server. This allows users to debug later on + // through commands such as "kubectl logs " and + // "kubectl describe pod ". Note that exited containers have terminated and + // therefore won't take CPU and memory resources. + // Otherwise, the executor pod is marked to be deleted from the API server. if (executorExited.exitCausedByApp) { logInfo(s"Executor $executorId exited because of the application.") deleteExecutorFromDataStructures(executorId) From 7afce3f1d61df2ecba3efd2019ca4d287fc0bffb Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 21 Nov 2017 11:18:00 -0800 Subject: [PATCH 20/29] Addressed more comments --- .../k8s/SparkKubernetesClientFactory.scala | 15 +++--- .../apache/spark/deploy/k8s/constants.scala | 47 ++++++++++--------- .../cluster/k8s/ExecutorPodFactory.scala | 4 +- .../KubernetesClusterSchedulerBackend.scala | 9 ++-- ...bernetesClusterSchedulerBackendSuite.scala | 9 ++-- .../spark/deploy/yarn/YarnAllocator.scala | 9 ++-- 6 files changed, 46 insertions(+), 47 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 38cda4565f45b..f167609edaf96 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -20,7 +20,7 @@ import java.io.File import com.google.common.base.Charsets import com.google.common.io.Files -import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient} +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient} import io.fabric8.kubernetes.client.utils.HttpClientUtils import okhttp3.Dispatcher @@ -88,16 +88,15 @@ private[spark] object SparkKubernetesClientFactory { new DefaultKubernetesClient(httpClientWithCustomDispatcher, config) } - private implicit class OptionConfigurableConfigBuilder(configBuilder: ConfigBuilder) { + private implicit class OptionConfigurableConfigBuilder(val configBuilder: ConfigBuilder) + extends AnyVal { def withOption[T] - (option: Option[T]) - (configurator: ((T, ConfigBuilder) => ConfigBuilder)): OptionConfigurableConfigBuilder = { - new OptionConfigurableConfigBuilder(option.map { opt => + (option: Option[T]) + (configurator: ((T, ConfigBuilder) => ConfigBuilder)): ConfigBuilder = { + option.map { opt => configurator(opt, configBuilder) - }.getOrElse(configBuilder)) + }.getOrElse(configBuilder) } - - def build(): Config = configBuilder.build() } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala index 664e645019820..b8ec18c8dc8e8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala @@ -16,34 +16,35 @@ */ package org.apache.spark.deploy.k8s -package object constants { +private[spark] object constants { + // Labels - private[spark] val SPARK_APP_ID_LABEL = "spark-app-selector" - private[spark] val SPARK_EXECUTOR_ID_LABEL = "spark-exec-id" - private[spark] val SPARK_ROLE_LABEL = "spark-role" - private[spark] val SPARK_POD_DRIVER_ROLE = "driver" - private[spark] val SPARK_POD_EXECUTOR_ROLE = "executor" + val SPARK_APP_ID_LABEL = "spark-app-selector" + val SPARK_EXECUTOR_ID_LABEL = "spark-exec-id" + val SPARK_ROLE_LABEL = "spark-role" + val SPARK_POD_DRIVER_ROLE = "driver" + val SPARK_POD_EXECUTOR_ROLE = "executor" // Default and fixed ports - private[spark] val DEFAULT_DRIVER_PORT = 7078 - private[spark] val DEFAULT_BLOCKMANAGER_PORT = 7079 - private[spark] val BLOCK_MANAGER_PORT_NAME = "blockmanager" - private[spark] val EXECUTOR_PORT_NAME = "executor" + val DEFAULT_DRIVER_PORT = 7078 + val DEFAULT_BLOCKMANAGER_PORT = 7079 + val BLOCK_MANAGER_PORT_NAME = "blockmanager" + val EXECUTOR_PORT_NAME = "executor" // Environment Variables - private[spark] val ENV_EXECUTOR_PORT = "SPARK_EXECUTOR_PORT" - private[spark] val ENV_DRIVER_URL = "SPARK_DRIVER_URL" - private[spark] val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES" - private[spark] val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY" - private[spark] val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID" - private[spark] val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID" - private[spark] val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP" - private[spark] val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH" - private[spark] val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH" - private[spark] val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_" + val ENV_EXECUTOR_PORT = "SPARK_EXECUTOR_PORT" + val ENV_DRIVER_URL = "SPARK_DRIVER_URL" + val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES" + val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY" + val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID" + val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID" + val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP" + val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH" + val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH" + val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_" // Miscellaneous - private[spark] val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" - private[spark] val MEMORY_OVERHEAD_FACTOR = 0.10 - private[spark] val MEMORY_OVERHEAD_MIN_MIB = 384L + val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" + val MEMORY_OVERHEAD_FACTOR = 0.10 + val MEMORY_OVERHEAD_MIN_MIB = 384L } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 764167a68c229..69c33b31572bf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -61,7 +61,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) require( !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL), s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + - s" Spark.") + " Spark.") require( !executorLabels.contains(SPARK_ROLE_LABEL), s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.") @@ -172,7 +172,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) } val executorContainer = new ContainerBuilder() - .withName(s"executor") + .withName("executor") .withImage(executorDockerImage) .withImagePullPolicy(dockerImagePullPolicy) .withNewResources() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 43249dcf42135..d1abd2ee62102 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -330,7 +330,7 @@ private[spark] class KubernetesClusterSchedulerBackend( if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) { log.warn(s"Executor with id $executorId was not marked as disconnected, but the" + s" watch received an event of type $action for this executor. The executor may" + - s" have failed to start in the first place and never registered with the driver.") + " have failed to start in the first place and never registered with the driver.") } disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) @@ -354,7 +354,7 @@ private[spark] class KubernetesClusterSchedulerBackend( private def getExecutorExitStatus(containerStatus: ContainerStatus): Int = { Option(containerStatus.getState).map { containerState => - Option(containerState.getTerminated).map {containerStateTerminated => + Option(containerState.getTerminated).map { containerStateTerminated => containerStateTerminated.getExitCode.intValue() }.getOrElse(UNKNOWN_EXIT_CODE) }.getOrElse(UNKNOWN_EXIT_CODE) @@ -403,8 +403,8 @@ private[spark] class KubernetesClusterSchedulerBackend( } private class KubernetesDriverEndpoint( - rpcEnv: RpcEnv, - sparkProperties: Seq[(String, String)]) + rpcEnv: RpcEnv, + sparkProperties: Seq[(String, String)]) extends DriverEndpoint(rpcEnv, sparkProperties) { override def onDisconnected(rpcAddress: RpcAddress): Unit = { @@ -427,4 +427,3 @@ private object KubernetesClusterSchedulerBackend { // and assume the executor failed for good, and attribute it to a framework fault. val MAX_EXECUTOR_LOST_REASON_CHECKS = 10 } - diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 1ca169217fc3f..0871fbda75411 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -39,8 +39,7 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{Register import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.ThreadUtils -class KubernetesClusterSchedulerBackendSuite - extends SparkFunSuite with BeforeAndAfter { +class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAndAfter { private val APP_ID = "test-spark-app" private val DRIVER_POD_NAME = "spark-driver-pod" @@ -49,7 +48,7 @@ class KubernetesClusterSchedulerBackendSuite private val SPARK_DRIVER_PORT = 7077 private val POD_ALLOCATION_INTERVAL = 60L private val DRIVER_URL = RpcEndpointAddress( - SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString private val FIRST_EXECUTOR_POD = new PodBuilder() .withNewMetadata() .withName("pod1") @@ -75,9 +74,9 @@ class KubernetesClusterSchedulerBackendSuite private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] private type LABELED_PODS = FilterWatchListDeletable[ - Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]] + Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]] private type IN_NAMESPACE_PODS = NonNamespaceOperation[ - Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] @Mock private var sparkContext: SparkContext = _ diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index a3f3718f245b8..506adb363aa90 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -22,15 +22,16 @@ import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import java.util.regex.Pattern -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.conf.YarnConfiguration import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} import scala.util.control.NonFatal +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.hadoop.yarn.conf.YarnConfiguration + import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ From b75b4136352d4606a41ce2b3fe1c7e31fdf71ffc Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 21 Nov 2017 11:47:42 -0800 Subject: [PATCH 21/29] Fixed Scala style error --- .../spark/deploy/k8s/{config.scala => Config.scala} | 2 +- .../deploy/k8s/{constants.scala => Constants.scala} | 2 +- .../spark/deploy/k8s/SparkKubernetesClientFactory.scala | 2 +- .../spark/scheduler/cluster/k8s/ExecutorPodFactory.scala | 4 ++-- .../scheduler/cluster/k8s/KubernetesClusterManager.scala | 4 ++-- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 4 ++-- .../scheduler/cluster/k8s/ExecutorPodFactorySuite.scala | 9 +++------ .../k8s/KubernetesClusterSchedulerBackendSuite.scala | 6 +++--- 8 files changed, 15 insertions(+), 18 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/{config.scala => Config.scala} (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/{constants.scala => Constants.scala} (98%) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index c6eff28d3b87b..7a1963e6ad9f1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -21,7 +21,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit -private[spark] object config extends Logging { +private[spark] object Config extends Logging { val KUBERNETES_NAMESPACE = ConfigBuilder("spark.kubernetes.namespace") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index b8ec18c8dc8e8..4ddeefb15a89d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.deploy.k8s -private[spark] object constants { +private[spark] object Constants { // Labels val SPARK_APP_ID_LABEL = "spark-app-selector" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index f167609edaf96..9f484bc1d3693 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -25,7 +25,7 @@ import io.fabric8.kubernetes.client.utils.HttpClientUtils import okhttp3.Dispatcher import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.util.ThreadUtils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 69c33b31572bf..6ef0bf7f05dc9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -21,9 +21,9 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.ConfigurationUtils -import org.apache.spark.deploy.k8s.config._ -import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 5e4bd7c41594e..68ca6a7622171 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -21,9 +21,9 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.SparkContext +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory -import org.apache.spark.deploy.k8s.config._ -import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.util.ThreadUtils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index d1abd2ee62102..ac62216efeeb4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -30,8 +30,8 @@ import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.SparkException -import org.apache.spark.deploy.k8s.config._ -import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index d1e64375a8b84..4fb312432dd91 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -19,18 +19,16 @@ package org.apache.spark.scheduler.cluster.k8s import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{Pod, _} -import io.fabric8.kubernetes.client.KubernetesClient import org.mockito.MockitoAnnotations import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.config._ -import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { private val driverPodName: String = "driver-pod" private val driverPodUid: String = "driver-uid" - private val driverUrl: String = "driver-url" private val executorPrefix: String = "base" private val executorImage: String = "executor-image" private val driverPod = new PodBuilder() @@ -54,7 +52,6 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, executorPrefix) .set(EXECUTOR_DOCKER_IMAGE, executorImage) } - private var kubernetesClient: KubernetesClient = _ test("basic executor pod has reasonable defaults") { val factory = new ExecutorPodFactoryImpl(baseConf) @@ -130,7 +127,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef ENV_EXECUTOR_PORT -> "10000") ++ additionalEnvVars assert(executor.getSpec.getContainers.size() === 1) - assert(executor.getSpec.getContainers.get(0).getEnv().size() === defaultEnvs.size) + assert(executor.getSpec.getContainers.get(0).getEnv.size() === defaultEnvs.size) val mapEnvs = executor.getSpec.getContainers.get(0).getEnv.asScala.map { x => (x.getName, x.getValue) }.toMap diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 0871fbda75411..f1a2ba7994be6 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -26,13 +26,13 @@ import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations} import org.mockito.Matchers.{any, eq => mockitoEq} import org.mockito.Mockito.{doNothing, never, times, verify, when} import org.scalatest.BeforeAndAfter -import org.scalatest.mock.MockitoSugar._ +import org.scalatest.mockito.MockitoSugar._ import scala.collection.JavaConverters._ import scala.concurrent.Future import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.deploy.k8s.config._ -import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.rpc._ import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor} From 3b587b4b4362f184b148c22522821ef7b163717e Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 21 Nov 2017 18:59:08 -0800 Subject: [PATCH 22/29] Removed unused parameter in parsePrefixedKeyValuePairs --- .../org/apache/spark/deploy/k8s/ConfigurationUtils.scala | 4 +--- .../spark/scheduler/cluster/k8s/ExecutorPodFactory.scala | 9 +++------ 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index 56444c16fd0ee..01717479fddd9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -27,13 +27,11 @@ private[spark] object ConfigurationUtils { * * @param sparkConf Spark configuration * @param prefix the given property name prefix - * @param configType a descriptive note on the type of entities of interest * @return a Map storing the configuration property keys and values */ def parsePrefixedKeyValuePairs( sparkConf: SparkConf, - prefix: String, - configType: String): Map[String, String] = { + prefix: String): Map[String, String] = { sparkConf.getAllWithPrefix(prefix).toMap } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 6ef0bf7f05dc9..3914d87758fcf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -53,8 +53,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, - KUBERNETES_EXECUTOR_LABEL_PREFIX, - "executor label") + KUBERNETES_EXECUTOR_LABEL_PREFIX) require( !executorLabels.contains(SPARK_APP_ID_LABEL), s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.") @@ -69,13 +68,11 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) private val executorAnnotations = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, - KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, - "executor annotation") + KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) private val nodeSelector = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, - KUBERNETES_NODE_SELECTOR_PREFIX, - "node selector") + KUBERNETES_NODE_SELECTOR_PREFIX) private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE) private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) From cb12fecb9cc8b6686b08ef1e82de3e62f32b4b73 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Wed, 22 Nov 2017 09:40:30 -0800 Subject: [PATCH 23/29] Another round of comments --- .../org/apache/spark/deploy/k8s/Config.scala | 5 ++- .../k8s/SparkKubernetesClientFactory.scala | 4 +- .../cluster/k8s/ExecutorPodFactory.scala | 2 +- .../KubernetesClusterSchedulerBackend.scala | 41 ++++++++++--------- 4 files changed, 28 insertions(+), 24 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 7a1963e6ad9f1..a621f3c861cbd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -40,8 +40,9 @@ private[spark] object Config extends Logging { val DOCKER_IMAGE_PULL_POLICY = ConfigBuilder("spark.kubernetes.docker.image.pullPolicy") - .doc("Docker image pull policy when pulling any docker image in Kubernetes integration") + .doc("Kubernetes image pull policy. Valid values are Always, Never, and IfNotPresent.") .stringConf + .checkValues(Set("Always", "Never", "IfNotPresent")) .createWithDefault("IfNotPresent") val APISERVER_AUTH_DRIVER_CONF_PREFIX = @@ -101,7 +102,7 @@ private[spark] object Config extends Logging { ConfigBuilder("spark.kubernetes.allocation.batch.delay") .doc("Number of seconds to wait between each round of executor allocation.") .longConf - .checkValue(value => value > 0, s"Allocation batch delay should be a positive integer") + .checkValue(value => value > 0, "Allocation batch delay should be a positive integer") .createWithDefault(1) val KUBERNETES_EXECUTOR_LIMIT_CORES = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 9f484bc1d3693..1e3f055e05766 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -92,8 +92,8 @@ private[spark] object SparkKubernetesClientFactory { extends AnyVal { def withOption[T] - (option: Option[T]) - (configurator: ((T, ConfigBuilder) => ConfigBuilder)): ConfigBuilder = { + (option: Option[T]) + (configurator: ((T, ConfigBuilder) => ConfigBuilder)): ConfigBuilder = { option.map { opt => configurator(opt, configBuilder) }.getOrElse(configBuilder) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 3914d87758fcf..afa95de0260fa 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -94,7 +94,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB private val executorCores = sparkConf.getDouble("spark.executor.cores", 1) - private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key) + private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES) override def createExecutorPod( executorId: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index ac62216efeeb4..47a8c189cd2eb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -103,24 +103,26 @@ private[spark] class KubernetesClusterSchedulerBackend( val currentTotalRegisteredExecutors = totalRegisteredExecutors.get val currentTotalExpectedExecutors = totalExpectedExecutors.get val currentNodeToLocalTaskCount = getNodesWithLocalTaskCounts() - if (currentTotalRegisteredExecutors < runningExecutorsToPods.size) { - logDebug("Waiting for pending executors before scaling") - } else if (currentTotalExpectedExecutors <= runningExecutorsToPods.size) { - logDebug("Maximum allowed executor limit reached. Not scaling up further.") - } else { - for (i <- 0 until math.min( - currentTotalExpectedExecutors - runningExecutorsToPods.size, podAllocationSize)) { - val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString - val executorPod = executorPodFactory.createExecutorPod( - executorId, - applicationId(), - driverUrl, - conf.getExecutorEnv, - driverPod, - currentNodeToLocalTaskCount) - executorsToAllocate(executorId) = executorPod - logInfo( - s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}") + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + if (currentTotalRegisteredExecutors < runningExecutorsToPods.size) { + logDebug("Waiting for pending executors before scaling") + } else if (currentTotalExpectedExecutors <= runningExecutorsToPods.size) { + logDebug("Maximum allowed executor limit reached. Not scaling up further.") + } else { + for (i <- 0 until math.min( + currentTotalExpectedExecutors - runningExecutorsToPods.size, podAllocationSize)) { + val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString + val executorPod = executorPodFactory.createExecutorPod( + executorId, + applicationId(), + driverUrl, + conf.getExecutorEnv, + driverPod, + currentNodeToLocalTaskCount) + executorsToAllocate(executorId) = executorPod + logInfo( + s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}") + } } } @@ -182,7 +184,8 @@ private[spark] class KubernetesClusterSchedulerBackend( def deleteExecutorFromClusterAndDataStructures(executorId: String): Unit = { deleteExecutorFromDataStructures(executorId).foreach { pod => - kubernetesClient.pods().delete(pod) } + kubernetesClient.pods().delete(pod) + } } def deleteExecutorFromDataStructures(executorId: String): Option[Pod] = { From ae396cf436aa9a2bbc9f73dcb0447a525457240c Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 27 Nov 2017 11:30:38 -0800 Subject: [PATCH 24/29] Addressed latest comments --- docs/configuration.md | 4 ++-- .../spark/scheduler/cluster/k8s/ExecutorPodFactory.scala | 9 --------- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 1 + 3 files changed, 3 insertions(+), 11 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 7a777d3c6fa3d..360286474d566 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1397,10 +1397,10 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.minRegisteredResourcesRatio - 0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained mode + 2.3.0 for KUBERNETES mode; 0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained mode The minimum ratio of registered resources (registered resources / total expected resources) - (resources are executors in yarn mode, CPU cores in standalone mode and Mesos coarsed-grained + (resources are executors in yarn mode and Kubernetes mode, CPU cores in standalone mode and Mesos coarsed-grained mode ['spark.cores.max' value is total expected resources for Mesos coarse-grained mode] ) to wait for before scheduling begins. Specified as a double between 0.0 and 1.0. Regardless of whether the minimum ratio of resources has been reached, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index afa95de0260fa..86bf35500383c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -46,8 +46,6 @@ private[spark] trait ExecutorPodFactory { private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) extends ExecutorPodFactory { - import ExecutorPodFactoryImpl._ - private val executorExtraClasspath = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) @@ -76,7 +74,6 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE) private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) - private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) private val blockManagerPort = sparkConf .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) @@ -139,7 +136,6 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) } }.getOrElse(Seq.empty[EnvVar]) val executorEnv = (Seq( - (ENV_EXECUTOR_PORT, executorPort.toString), (ENV_DRIVER_URL, driverUrl), // Executor backend expects integral value for executor cores, so round it up to an int. (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString), @@ -159,7 +155,6 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) .build() ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq val requiredPorts = Seq( - (EXECUTOR_PORT_NAME, executorPort), (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) .map { case (name, port) => new ContainerPortBuilder() @@ -220,7 +215,3 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) .build() } } - -private object ExecutorPodFactoryImpl { - private val DEFAULT_STATIC_PORT = 10000 -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 47a8c189cd2eb..36c948deb2ca3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -224,6 +224,7 @@ private[spark] class KubernetesClusterSchedulerBackend( override def stop(): Unit = { // stop allocation of new resources and caches. allocatorExecutor.shutdown() + allocatorExecutor.awaitTermination(30, TimeUnit.SECONDS) // send stop message to executors so they shut down cleanly super.stop() From f8e3249d7321bd0d950c252919cb499bed571af9 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 27 Nov 2017 11:55:13 -0800 Subject: [PATCH 25/29] Addressed comments around licensing on new dependencies --- NOTICE | 6 ++++++ resource-managers/kubernetes/core/pom.xml | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/NOTICE b/NOTICE index f4b64b5c3f470..6ec240efbf12e 100644 --- a/NOTICE +++ b/NOTICE @@ -448,6 +448,12 @@ Copyright (C) 2011 Google Inc. Apache Commons Pool Copyright 1999-2009 The Apache Software Foundation +This product includes/uses Kubernetes & OpenShift 3 Java Client (https://github.com/fabric8io/kubernetes-client) +Copyright (C) 2015 Red Hat, Inc. + +This product includes/uses OkHttp (https://github.com/square/okhttp) +Copyright (C) 2012 The Android Open Source Project + ========================================================================= == NOTICE file corresponding to section 4(d) of the Apache License, == == Version 2.0, in this case for the DataNucleus distribution. == diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 204fdc7fa4afb..7d35aea8a4142 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -83,6 +83,12 @@ test + + com.squareup.okhttp3 + okhttp + 3.8.1 + + From a44c29e9effe3698b5cf948a0fbb12ab16282670 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 27 Nov 2017 14:17:13 -0800 Subject: [PATCH 26/29] Fixed unit tests and made maximum executor lost reason checks configurable --- .../main/scala/org/apache/spark/deploy/k8s/Config.scala | 9 +++++++++ .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 8 ++++---- .../scheduler/cluster/k8s/ExecutorPodFactorySuite.scala | 3 +-- .../k8s/KubernetesClusterSchedulerBackendSuite.scala | 4 +++- 4 files changed, 17 insertions(+), 7 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index a621f3c861cbd..f8e21546ab841 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -111,5 +111,14 @@ private[spark] object Config extends Logging { .stringConf .createOptional + val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS = + ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts") + .doc("Maximum number of attempts allowed for checking the reason of an executor loss " + + "before it is assumed that the executor failed.") + .intConf + .checkValue(value => value > 0, "Maximum attempts of checks of executor lost reason " + + "must be a positive integer") + .createWithDefault(5) + val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 36c948deb2ca3..6ca9a9fd1ba5b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -90,6 +90,9 @@ private[spark] class KubernetesClusterSchedulerBackend( private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) + private val executorLostReasonCheckMaxAttempts = conf.get( + KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS) + private val allocatorRunnable = new Runnable { // Maintains a map of executor id to count of checks performed to learn the loss reason @@ -174,7 +177,7 @@ private[spark] class KubernetesClusterSchedulerBackend( def removeExecutorOrIncrementLossReasonCheckCount(executorId: String): Unit = { val reasonCheckCount = executorReasonCheckAttemptCounts.getOrElse(executorId, 0) - if (reasonCheckCount >= MAX_EXECUTOR_LOST_REASON_CHECKS) { + if (reasonCheckCount >= executorLostReasonCheckMaxAttempts) { removeExecutor(executorId, SlaveLost("Executor lost for unknown reasons.")) deleteExecutorFromClusterAndDataStructures(executorId) } else { @@ -427,7 +430,4 @@ private[spark] class KubernetesClusterSchedulerBackend( private object KubernetesClusterSchedulerBackend { private val UNKNOWN_EXIT_CODE = -1 - // Number of times we are allowed check for the loss reason for an executor before we give up - // and assume the executor failed for good, and attribute it to a framework fault. - val MAX_EXECUTOR_LOST_REASON_CHECKS = 10 } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index 4fb312432dd91..1c7717c238096 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -123,8 +123,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef ENV_EXECUTOR_CORES -> "1", ENV_EXECUTOR_MEMORY -> "1g", ENV_APPLICATION_ID -> "dummy", - ENV_EXECUTOR_POD_IP -> null, - ENV_EXECUTOR_PORT -> "10000") ++ additionalEnvVars + ENV_EXECUTOR_POD_IP -> null) ++ additionalEnvVars assert(executor.getSpec.getContainers.size() === 1) assert(executor.getSpec.getContainers.get(0).getEnv.size() === defaultEnvs.size) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index f1a2ba7994be6..3febb2f47cfd4 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -330,6 +330,8 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn sparkConf .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val executorLostReasonCheckMaxAttempts = sparkConf.get( + KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS) val scheduler = newSchedulerBackend() scheduler.start() @@ -346,7 +348,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn .apply(registerFirstExecutorMessage) driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) - 1 to KubernetesClusterSchedulerBackend.MAX_EXECUTOR_LOST_REASON_CHECKS foreach { _ => + 1 to executorLostReasonCheckMaxAttempts foreach { _ => allocatorRunnable.getValue.run() verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) } From 4bed817e5ab1a26e70050254fec453e839af4c8c Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 27 Nov 2017 14:54:47 -0800 Subject: [PATCH 27/29] Removed default value for executor Docker image --- .../src/main/scala/org/apache/spark/deploy/k8s/Config.scala | 3 +-- .../spark/scheduler/cluster/k8s/ExecutorPodFactory.scala | 6 ++++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index f8e21546ab841..a7fb1f8ce2609 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -16,7 +16,6 @@ */ package org.apache.spark.deploy.k8s -import org.apache.spark.SPARK_VERSION import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit @@ -36,7 +35,7 @@ private[spark] object Config extends Logging { .doc("Docker image to use for the executors. Specify this using the standard Docker tag" + " format.") .stringConf - .createWithDefault(s"spark-executor:$SPARK_VERSION") + .createOptional val DOCKER_IMAGE_PULL_POLICY = ConfigBuilder("spark.kubernetes.docker.image.pullPolicy") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 86bf35500383c..f79155b117b67 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -20,7 +20,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ @@ -72,7 +72,9 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) - private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE) + private val executorDockerImage = sparkConf + .get(EXECUTOR_DOCKER_IMAGE) + .getOrElse(throw new SparkException("Must specify the executor Docker image")) private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) private val blockManagerPort = sparkConf .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) From c3861864efbde824beb722d0cdfe861277086c48 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 27 Nov 2017 15:50:14 -0800 Subject: [PATCH 28/29] Close the executor pod watcher before deleting the executor pods --- .../KubernetesClusterSchedulerBackend.scala | 29 ++++++++++++------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 6ca9a9fd1ba5b..91bd9081af19a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -112,7 +112,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } else if (currentTotalExpectedExecutors <= runningExecutorsToPods.size) { logDebug("Maximum allowed executor limit reached. Not scaling up further.") } else { - for (i <- 0 until math.min( + for (_ <- 0 until math.min( currentTotalExpectedExecutors - runningExecutorsToPods.size, podAllocationSize)) { val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString val executorPod = executorPodFactory.createExecutorPod( @@ -232,19 +232,19 @@ private[spark] class KubernetesClusterSchedulerBackend( // send stop message to executors so they shut down cleanly super.stop() - // then delete the executor pods - Utils.tryLogNonFatalError { - val executorPodsToDelete = RUNNING_EXECUTOR_PODS_LOCK.synchronized { - val runningExecutorPodsCopy = Seq(runningExecutorsToPods.values.toSeq: _*) - runningExecutorsToPods.clear() - runningExecutorPodsCopy - } - kubernetesClient.pods().delete(executorPodsToDelete: _*) - executorPodsByIPs.clear() + try { val resource = executorWatchResource.getAndSet(null) if (resource != null) { resource.close() } + } catch { + case e: Throwable => logWarning("Failed to close the executor pod watcher", e) + } + + // then delete the executor pods + Utils.tryLogNonFatalError { + deleteExecutorPodsOnStop() + executorPodsByIPs.clear() } Utils.tryLogNonFatalError { logInfo("Closing kubernetes client") @@ -298,6 +298,15 @@ private[spark] class KubernetesClusterSchedulerBackend( true } + private def deleteExecutorPodsOnStop(): Unit = { + val executorPodsToDelete = RUNNING_EXECUTOR_PODS_LOCK.synchronized { + val runningExecutorPodsCopy = Seq(runningExecutorsToPods.values.toSeq: _*) + runningExecutorsToPods.clear() + runningExecutorPodsCopy + } + kubernetesClient.pods().delete(executorPodsToDelete: _*) + } + private class ExecutorPodsWatcher extends Watcher[Pod] { private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1 From b85cfc4038c8de9340b78d10edf88ab76dd90ba3 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 27 Nov 2017 16:05:09 -0800 Subject: [PATCH 29/29] Addressed more comments --- docs/configuration.md | 2 +- .../org/apache/spark/deploy/k8s/Config.scala | 26 +++++++++---------- .../KubernetesClusterSchedulerBackend.scala | 10 +++---- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 360286474d566..7129b904698a2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1397,7 +1397,7 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.minRegisteredResourcesRatio - 2.3.0 for KUBERNETES mode; 0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained mode + 0.8 for KUBERNETES mode; 0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained mode The minimum ratio of registered resources (registered resources / total expected resources) (resources are executors in yarn mode and Kubernetes mode, CPU cores in standalone mode and Mesos coarsed-grained diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index a7fb1f8ce2609..f0742b91987b6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -24,16 +24,16 @@ private[spark] object Config extends Logging { val KUBERNETES_NAMESPACE = ConfigBuilder("spark.kubernetes.namespace") - .doc("The namespace that will be used for running the driver and executor pods. When using" + - " spark-submit in cluster mode, this can also be passed to spark-submit via the" + - " --kubernetes-namespace command line argument.") + .doc("The namespace that will be used for running the driver and executor pods. When using " + + "spark-submit in cluster mode, this can also be passed to spark-submit via the " + + "--kubernetes-namespace command line argument.") .stringConf .createWithDefault("default") val EXECUTOR_DOCKER_IMAGE = ConfigBuilder("spark.kubernetes.executor.docker.image") - .doc("Docker image to use for the executors. Specify this using the standard Docker tag" + - " format.") + .doc("Docker image to use for the executors. Specify this using the standard Docker tag " + + "format.") .stringConf .createOptional @@ -56,10 +56,10 @@ private[spark] object Config extends Logging { val KUBERNETES_SERVICE_ACCOUNT_NAME = ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName") - .doc("Service account that is used when running the driver pod. The driver pod uses" + - " this service account when requesting executor pods from the API server. If specific" + - " credentials are given for the driver pod to use, the driver will favor" + - " using those credentials instead.") + .doc("Service account that is used when running the driver pod. The driver pod uses " + + "this service account when requesting executor pods from the API server. If specific " + + "credentials are given for the driver pod to use, the driver will favor " + + "using those credentials instead.") .stringConf .createOptional @@ -68,9 +68,9 @@ private[spark] object Config extends Logging { // based on the executor memory. val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.kubernetes.executor.memoryOverhead") - .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" + - " is memory that accounts for things like VM overheads, interned strings, other native" + - " overheads, etc. This tends to grow with the executor size. (typically 6-10%).") + .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This " + + "is memory that accounts for things like VM overheads, interned strings, other native " + + "overheads, etc. This tends to grow with the executor size. (typically 6-10%).") .bytesConf(ByteUnit.MiB) .createOptional @@ -117,7 +117,7 @@ private[spark] object Config extends Logging { .intConf .checkValue(value => value > 0, "Maximum attempts of checks of executor lost reason " + "must be a positive integer") - .createWithDefault(5) + .createWithDefault(10) val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 91bd9081af19a..e79c987852db2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -344,9 +344,9 @@ private[spark] class KubernetesClusterSchedulerBackend( podsWithKnownExitReasons.put(pod.getMetadata.getName, executorExitReason) if (!disconnectedPodsByExecutorIdPendingRemoval.containsKey(executorId)) { - log.warn(s"Executor with id $executorId was not marked as disconnected, but the" + - s" watch received an event of type $action for this executor. The executor may" + - " have failed to start in the first place and never registered with the driver.") + log.warn(s"Executor with id $executorId was not marked as disconnected, but the " + + s"watch received an event of type $action for this executor. The executor may " + + "have failed to start in the first place and never registered with the driver.") } disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) @@ -388,8 +388,8 @@ private[spark] class KubernetesClusterSchedulerBackend( // container was probably actively killed by the driver. if (isPodAlreadyReleased(pod)) { ExecutorExited(containerExitStatus, exitCausedByApp = false, - s"Container in pod ${pod.getMetadata.getName} exited from explicit termination" + - " request.") + s"Container in pod ${pod.getMetadata.getName} exited from explicit termination " + + "request.") } else { val containerExitReason = s"Pod ${pod.getMetadata.getName}'s executor container " + s"exited with exit status code $containerExitStatus."