From 9979423db32301543359abf60effc0ef742adc35 Mon Sep 17 00:00:00 2001 From: Ben Barnard Date: Tue, 6 Jun 2017 03:19:35 +0200 Subject: [PATCH] Add support for Nomad as a scheduler backend --- .travis.yml | 2 +- assembly/pom.xml | 10 + .../apache/spark/tags/ExtendedNomadTest.java | 30 + .../org/apache/spark/SecurityManager.scala | 21 +- .../org/apache/spark/deploy/SparkSubmit.scala | 83 +- .../spark/deploy/SparkSubmitArguments.scala | 76 +- .../CoarseGrainedSchedulerBackend.scala | 14 +- .../status/api/v1/JacksonMessageWriter.scala | 2 +- dev/create-release/release-build.sh | 6 +- dev/deps/spark-deps-hadoop-2.6 | 17 +- dev/deps/spark-deps-hadoop-2.7 | 17 +- dev/lint-java | 2 +- dev/mima | 2 +- dev/scalastyle | 1 + dev/test-dependencies.sh | 2 +- docs/_layouts/global.html | 1 + docs/building-spark.md | 10 +- docs/cluster-overview.md | 7 +- docs/configuration.md | 12 +- docs/img/nomad-client-mode.png | Bin 0 -> 28213 bytes docs/img/nomad-cluster-mode.png | Bin 0 -> 25001 bytes docs/index.md | 8 +- docs/job-scheduling.md | 15 +- docs/rdd-programming-guide.md | 2 +- docs/running-on-nomad.md | 722 ++++++++++++++++++ docs/security.md | 2 +- docs/streaming-flume-integration.md | 2 +- .../launcher/AbstractCommandBuilder.java | 1 + .../launcher/SparkClassCommandBuilder.java | 5 + .../launcher/SparkSubmitOptionParser.java | 20 +- pom.xml | 10 +- project/SparkBuild.scala | 4 +- resource-managers/nomad/pom.xml | 96 +++ ...che.spark.scheduler.ExternalClusterManager | 1 + .../deploy/nomad/ApplicationRunCommand.scala | 26 + .../deploy/nomad/NomadClusterModeConf.scala | 90 +++ .../nomad/NomadClusterModeLauncher.scala | 218 ++++++ .../spark/deploy/nomad/NomadDryRun.scala | 45 ++ .../spark/executor/NomadExecutorBackend.scala | 36 + .../apache/spark/nomad/EvaluationUtils.scala | 88 +++ .../org/apache/spark/nomad/JobUtils.scala | 248 ++++++ .../scheduler/cluster/nomad/DriverTask.scala | 135 ++++ .../cluster/nomad/DriverTaskGroup.scala | 36 + .../cluster/nomad/ExecutorTask.scala | 97 +++ .../cluster/nomad/ExecutorTaskGroup.scala | 65 ++ .../cluster/nomad/NomadClusterManager.scala | 64 ++ .../nomad/NomadClusterManagerConf.scala | 154 ++++ .../nomad/NomadClusterSchedulerBackend.scala | 189 +++++ .../cluster/nomad/NomadJobManipulator.scala | 113 +++ .../cluster/nomad/ShuffleServiceTask.scala | 56 ++ .../cluster/nomad/SparkNomadJob.scala | 305 ++++++++ .../nomad/SparkNomadJobController.scala | 100 +++ .../nomad/SparkNomadTaskGroupType.scala | 96 +++ .../cluster/nomad/SparkNomadTaskType.scala | 304 ++++++++ .../nomad/src/test/resources/log4j.properties | 31 + .../scheduler/cluster/nomad/job-template.json | 182 +++++ .../cluster/nomad/job-template.nomad | 84 ++ .../deploy/nomad/BaseNomadClusterSuite.scala | 318 ++++++++ .../spark/deploy/nomad/HttpTestServer.scala | 104 +++ .../spark/launcher/TestClasspathBuilder.scala | 36 + .../cluster/nomad/NomadClusterSuite.scala | 389 ++++++++++ .../cluster/nomad/NomadPythonSuite.scala | 102 +++ resource-managers/nomad/test-apps/pom.xml | 57 ++ .../nomad/ConfigurationCheckingSparkApp.scala | 78 ++ .../nomad/DiskPersistedRddTestDriver.scala | 47 ++ .../nomad/EnvironmentPostingTestDriver.scala | 52 ++ .../nomad/FileDistributionTestDriver.scala | 47 ++ .../cluster/nomad/LocalDiskWritingApp.scala | 40 + .../nomad/ResourceUploaderTestApp.scala | 48 ++ .../cluster/nomad/SaveExecutorInfo.scala | 36 + .../cluster/nomad/SaveToDiskTestApp.scala | 39 + .../cluster/nomad/StreamingApp.scala | 66 ++ .../cluster/nomad/TestApplication.scala | 101 +++ .../ThrowExceptionAfterContextInit.scala | 32 + .../cluster/YarnSchedulerBackend.scala | 8 + 75 files changed, 5475 insertions(+), 90 deletions(-) create mode 100644 common/tags/src/test/java/org/apache/spark/tags/ExtendedNomadTest.java create mode 100644 docs/img/nomad-client-mode.png create mode 100644 docs/img/nomad-cluster-mode.png create mode 100644 docs/running-on-nomad.md create mode 100644 resource-managers/nomad/pom.xml create mode 100644 resource-managers/nomad/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/ApplicationRunCommand.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadClusterModeConf.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadClusterModeLauncher.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadDryRun.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/executor/NomadExecutorBackend.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/nomad/EvaluationUtils.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/nomad/JobUtils.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DriverTask.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DriverTaskGroup.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ExecutorTask.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ExecutorTaskGroup.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterManager.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterManagerConf.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterSchedulerBackend.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadJobManipulator.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ShuffleServiceTask.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadJob.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadJobController.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadTaskGroupType.scala create mode 100644 resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadTaskType.scala create mode 100644 resource-managers/nomad/src/test/resources/log4j.properties create mode 100644 resource-managers/nomad/src/test/resources/org/apache/spark/scheduler/cluster/nomad/job-template.json create mode 100644 resource-managers/nomad/src/test/resources/org/apache/spark/scheduler/cluster/nomad/job-template.nomad create mode 100644 resource-managers/nomad/src/test/scala/org/apache/spark/deploy/nomad/BaseNomadClusterSuite.scala create mode 100644 resource-managers/nomad/src/test/scala/org/apache/spark/deploy/nomad/HttpTestServer.scala create mode 100644 resource-managers/nomad/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala create mode 100644 resource-managers/nomad/src/test/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterSuite.scala create mode 100644 resource-managers/nomad/src/test/scala/org/apache/spark/scheduler/cluster/nomad/NomadPythonSuite.scala create mode 100644 resource-managers/nomad/test-apps/pom.xml create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ConfigurationCheckingSparkApp.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DiskPersistedRddTestDriver.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/EnvironmentPostingTestDriver.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/FileDistributionTestDriver.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/LocalDiskWritingApp.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ResourceUploaderTestApp.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SaveExecutorInfo.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SaveToDiskTestApp.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/StreamingApp.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/TestApplication.scala create mode 100644 resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ThrowExceptionAfterContextInit.scala diff --git a/.travis.yml b/.travis.yml index d7e9f8c0290e8..7d321a0ad17c1 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 -Pmesos -Pnomad -Pyarn -Pkinesis-asl -Phive -Phive-thriftserver install # 6. Run lint-java. script: diff --git a/assembly/pom.xml b/assembly/pom.xml index 464af16e46f6e..492afbe7498cf 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -148,6 +148,16 @@ + + nomad + + + org.apache.spark + spark-nomad_${scala.binary.version} + ${project.version} + + + hive diff --git a/common/tags/src/test/java/org/apache/spark/tags/ExtendedNomadTest.java b/common/tags/src/test/java/org/apache/spark/tags/ExtendedNomadTest.java new file mode 100644 index 0000000000000..b1180bd9149e5 --- /dev/null +++ b/common/tags/src/test/java/org/apache/spark/tags/ExtendedNomadTest.java @@ -0,0 +1,30 @@ +/* + * 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.tags; + +import org.scalatest.TagAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@TagAnnotation +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface ExtendedNomadTest { } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 2480e56b72ccf..7a6b055b929cb 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -426,6 +426,16 @@ private[spark] class SecurityManager( * we throw an exception. */ private def generateSecretKey(): String = { + + def generate(): String = { + val rnd = new SecureRandom() + val length = sparkConf.getInt("spark.authenticate.secretBitLength", 256) / JByte.SIZE + val secret = new Array[Byte](length) + rnd.nextBytes(secret) + + HashCodes.fromBytes(secret).toString + } + if (!isAuthenticationEnabled) { null } else if (SparkHadoopUtil.get.isYarnMode) { @@ -435,12 +445,7 @@ private[spark] class SecurityManager( val secretKey = SparkHadoopUtil.get.getSecretKeyFromUserCredentials(SECRET_LOOKUP_KEY) if (secretKey == null || secretKey.length == 0) { logDebug("generateSecretKey: yarn mode, secret key from credentials is null") - val rnd = new SecureRandom() - val length = sparkConf.getInt("spark.authenticate.secretBitLength", 256) / JByte.SIZE - val secret = new Array[Byte](length) - rnd.nextBytes(secret) - - val cookie = HashCodes.fromBytes(secret).toString() + val cookie = generate() SparkHadoopUtil.get.addSecretKeyToUserCredentials(SECRET_LOOKUP_KEY, cookie) cookie } else { @@ -452,6 +457,10 @@ private[spark] class SecurityManager( Option(sparkConf.getenv(SecurityManager.ENV_AUTH_SECRET)) .orElse(sparkConf.getOption(SecurityManager.SPARK_AUTH_SECRET_CONF)) match { case Some(value) => value + case None if sparkConf.getOption("spark.master").exists(_.startsWith("nomad")) => + val secret = generate() + sparkConf.set(SecurityManager.SPARK_AUTH_SECRET_CONF, secret) + secret case None => throw new IllegalArgumentException( "Error: a secret key must be specified via the " + diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index d13fb4193970b..959a7e4f74364 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy import java.io.{File, IOException} import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} -import java.net.URL +import java.net.{URI, URL} import java.nio.file.Files import java.security.PrivilegedExceptionAction import java.text.ParseException @@ -72,7 +72,8 @@ object SparkSubmit extends CommandLineUtils { private val STANDALONE = 2 private val MESOS = 4 private val LOCAL = 8 - private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL + private val NOMAD = 16 + private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL | NOMAD // Deploy modes private val CLIENT = 1 @@ -232,9 +233,10 @@ object SparkSubmit extends CommandLineUtils { YARN case m if m.startsWith("spark") => STANDALONE case m if m.startsWith("mesos") => MESOS + case m if m.startsWith("nomad") => NOMAD case m if m.startsWith("local") => LOCAL case _ => - printErrorAndExit("Master must either be yarn or start with spark, mesos, local") + printErrorAndExit("Master must either be yarn or start with spark, mesos, nomad, local") -1 } @@ -277,6 +279,7 @@ object SparkSubmit extends CommandLineUtils { } val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER + val isNomadCluster = clusterManager == NOMAD && deployMode == CLUSTER // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files // too for packages that include Python code @@ -322,7 +325,8 @@ object SparkSubmit extends CommandLineUtils { // Require all python files to be local, so we can add them to the PYTHONPATH // In YARN cluster mode, python files are distributed as regular files, which can be non-local. // In Mesos cluster mode, non-local python files are automatically downloaded by Mesos. - if (args.isPython && !isYarnCluster && !isMesosCluster) { + // In Nomad cluster mode, non-local python files are automatically downloaded by Nomad. + if (args.isPython && !isYarnCluster && !isMesosCluster && !isNomadCluster) { if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { printErrorAndExit(s"Only local python files are supported: ${args.primaryResource}") } @@ -333,7 +337,7 @@ object SparkSubmit extends CommandLineUtils { } // Require all R files to be local - if (args.isR && !isYarnCluster && !isMesosCluster) { + if (args.isR && !isYarnCluster && !isMesosCluster && !isNomadCluster) { if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { printErrorAndExit(s"Only local R files are supported: ${args.primaryResource}") } @@ -423,6 +427,11 @@ object SparkSubmit extends CommandLineUtils { printErrorAndExit("Distributing R packages with mesos cluster is not supported.") } + // TODO: Support distributing R packages with Nomad cluster + if (args.isR && clusterManager == NOMAD && !RUtils.rPackages.isEmpty) { + printErrorAndExit("Distributing R packages with Nomad cluster is not supported.") + } + // If we're running an R app, set the main class to our specific R runner if (args.isR && deployMode == CLIENT) { if (args.primaryResource == SPARKR_SHELL) { @@ -466,28 +475,41 @@ object SparkSubmit extends CommandLineUtils { // Yarn only OptionAssigner(args.queue, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.queue"), - OptionAssigner(args.numExecutors, YARN, ALL_DEPLOY_MODES, - sysProp = "spark.executor.instances"), OptionAssigner(args.jars, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.jars"), OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.files"), OptionAssigner(args.archives, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.archives"), OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.principal"), OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.keytab"), + // Nomad only + OptionAssigner(args.distribution, NOMAD, ALL_DEPLOY_MODES, + sysProp = "spark.nomad.sparkDistribution"), + OptionAssigner(args.dockerImage, NOMAD, ALL_DEPLOY_MODES, + sysProp = "spark.nomad.dockerImage"), + OptionAssigner(args.executorCpu, NOMAD, ALL_DEPLOY_MODES, sysProp = "spark.executor.cpu"), + OptionAssigner(args.driverCpu, NOMAD, CLUSTER, sysProp = "spark.driver.cpu"), + OptionAssigner(args.monitorUntil, NOMAD, CLUSTER, + sysProp = "spark.nomad.cluster.monitorUntil"), + OptionAssigner(args.nomadTemplate, NOMAD, ALL_DEPLOY_MODES, + sysProp = "spark.nomad.job.template"), + // Other options - OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES, + OptionAssigner(args.numExecutors, YARN | NOMAD, ALL_DEPLOY_MODES, + sysProp = "spark.executor.instances"), + OptionAssigner(args.executorCores, STANDALONE | YARN | NOMAD, ALL_DEPLOY_MODES, sysProp = "spark.executor.cores"), - OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, + OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | NOMAD, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.cores.max"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS | NOMAD, ALL_DEPLOY_MODES, sysProp = "spark.files"), OptionAssigner(args.jars, LOCAL, CLIENT, sysProp = "spark.jars"), - OptionAssigner(args.jars, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.jars"), - OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN, CLUSTER, + OptionAssigner(args.jars, STANDALONE | MESOS | NOMAD, ALL_DEPLOY_MODES, + sysProp = "spark.jars"), + OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN | NOMAD, CLUSTER, sysProp = "spark.driver.memory"), - OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN, CLUSTER, + OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN | NOMAD, CLUSTER, sysProp = "spark.driver.cores"), OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, sysProp = "spark.driver.supervise"), @@ -622,6 +644,33 @@ object SparkSubmit extends CommandLineUtils { } } + if (isNomadCluster || (clusterManager == NOMAD && args.isNomadDryRun)) { + childMainClass = + if (args.isNomadDryRun) "org.apache.spark.deploy.nomad.NomadDryRun" + else "org.apache.spark.deploy.nomad.NomadClusterModeLauncher" + if (args.driverExtraClassPath != null) { + childClasspath ++= args.driverExtraClassPath.split(":") + } + if (args.isPython) { + childArgs += ("--primary-py-file", args.primaryResource) + childArgs += ("--class", "org.apache.spark.deploy.PythonRunner") + if (args.pyFiles != null) { + sysProps("spark.submit.pyFiles") = args.pyFiles + } + } else if (args.isR) { + childArgs += ("--primary-r-file", args.primaryResource) + childArgs += ("--class", "org.apache.spark.deploy.RRunner") + } else { + if (args.primaryResource != SparkLauncher.NO_RESOURCE) { + childArgs += ("--jar", args.primaryResource) + } + childArgs += ("--class", args.mainClass) + } + if (args.childArgs != null) { + args.childArgs.foreach { arg => childArgs += ("--arg", arg) } + } + } + // Load any properties specified through --conf and the default properties file for ((k, v) <- args.sparkProperties) { sysProps.getOrElseUpdate(k, v) @@ -651,10 +700,10 @@ object SparkSubmit extends CommandLineUtils { // explicitly sets `spark.submit.pyFiles` in his/her default properties file. sysProps.get("spark.submit.pyFiles").foreach { pyFiles => val resolvedPyFiles = Utils.resolveURIs(pyFiles) - val formattedPyFiles = if (!isYarnCluster && !isMesosCluster) { + val formattedPyFiles = if (!isYarnCluster && !isMesosCluster && !isNomadCluster) { PythonRunner.formatPaths(resolvedPyFiles).mkString(",") } else { - // Ignoring formatting python path in yarn and mesos cluster mode, these two modes + // Ignoring formatting python path in yarn, mesos and nomad cluster mode, these modes // support dealing with remote python files, they could distribute and add python files // locally. resolvedPyFiles @@ -812,14 +861,14 @@ object SparkSubmit extends CommandLineUtils { * Return whether the given primary resource requires running python. */ private[deploy] def isPython(res: String): Boolean = { - res != null && res.endsWith(".py") || res == PYSPARK_SHELL + res != null && new URI(res).getPath.endsWith(".py") || res == PYSPARK_SHELL } /** * Return whether the given primary resource requires running R. */ private[deploy] def isR(res: String): Boolean = { - res != null && res.endsWith(".R") || res == SPARKR_SHELL + res != null && new URI(res).getPath.endsWith(".R") || res == SPARKR_SHELL } private[deploy] def isInternal(res: String): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index b76a3d2bea4c7..ef143c482df70 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -48,6 +48,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S var totalExecutorCores: String = null var propertiesFile: String = null var driverMemory: String = null + var driverCores: String = null var driverExtraClassPath: String = null var driverExtraLibraryPath: String = null var driverExtraJavaOptions: String = null @@ -76,11 +77,19 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S // Standalone cluster mode only var supervise: Boolean = false - var driverCores: String = null var submissionToKill: String = null var submissionToRequestStatusFor: String = null var useRest: Boolean = true // used internally + // Nomad mode only + var distribution: String = null + var dockerImage: String = null + var driverCpu: String = null + var executorCpu: String = null + var monitorUntil: String = null + var isNomadDryRun: Boolean = false + var nomadTemplate: String = null + /** Default properties present in the currently defined defaults file. */ lazy val defaultSparkProperties: HashMap[String, String] = { val defaultProperties = new HashMap[String, String]() @@ -169,6 +178,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S driverCores = Option(driverCores) .orElse(sparkProperties.get("spark.driver.cores")) .orNull + driverCpu = Option(driverCpu) + .orElse(sparkProperties.get("spark.driver.cpu")) + .orNull executorMemory = Option(executorMemory) .orElse(sparkProperties.get("spark.executor.memory")) .orElse(env.get("SPARK_EXECUTOR_MEMORY")) @@ -177,6 +189,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orElse(sparkProperties.get("spark.executor.cores")) .orElse(env.get("SPARK_EXECUTOR_CORES")) .orNull + executorCpu = Option(executorCpu) + .orElse(sparkProperties.get("spark.executor.cpu")) + .orNull totalExecutorCores = Option(totalExecutorCores) .orElse(sparkProperties.get("spark.cores.max")) .orNull @@ -198,6 +213,11 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S queue = Option(queue).orElse(sparkProperties.get("spark.yarn.queue")).orNull keytab = Option(keytab).orElse(sparkProperties.get("spark.yarn.keytab")).orNull principal = Option(principal).orElse(sparkProperties.get("spark.yarn.principal")).orNull + distribution = Option(distribution).orElse(sparkProperties.get("spark.nomad.sparkDistribution")) + .orNull + dockerImage = Option(dockerImage).orElse(sparkProperties.get("spark.nomad.dockerImage")).orNull + monitorUntil = Option(monitorUntil) + .orElse(sparkProperties.get("spark.nomad.cluster.monitorUntil")).orNull // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && !isR && primaryResource != null) { @@ -232,7 +252,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S // Set name from main class if not given name = Option(name).orElse(Option(mainClass)).orNull if (name == null && primaryResource != null) { - name = Utils.stripDirectory(primaryResource) + name = Utils.stripDirectory(new URI(primaryResource).getPath) } // Action should be SUBMIT unless otherwise specified @@ -322,10 +342,12 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | deployMode $deployMode | executorMemory $executorMemory | executorCores $executorCores + | executorCpu $executorCpu | totalExecutorCores $totalExecutorCores | propertiesFile $propertiesFile | driverMemory $driverMemory | driverCores $driverCores + | driverCpu $driverCpu | driverExtraClassPath $driverExtraClassPath | driverExtraLibraryPath $driverExtraLibraryPath | driverExtraJavaOptions $driverExtraJavaOptions @@ -378,6 +400,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case EXECUTOR_CORES => executorCores = value + case EXECUTOR_CPU => + executorCpu = value + case EXECUTOR_MEMORY => executorMemory = value @@ -387,6 +412,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case DRIVER_CORES => driverCores = value + case DRIVER_CPU => + driverCpu = value + case DRIVER_CLASS_PATH => driverExtraClassPath = value @@ -465,6 +493,21 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case USAGE_ERROR => printUsageAndExit(1) + case DISTRIBUTION => + distribution = value + + case DOCKER_IMAGE => + dockerImage = value + + case MONITOR_UNTIL => + monitorUntil = value + + case NOMAD_DRY_RUN => + isNomadDryRun = true + + case NOMAD_TEMPLATE => + nomadTemplate = value + case _ => throw new IllegalArgumentException(s"Unexpected argument '$opt'.") } @@ -514,7 +557,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S outStream.println( s""" |Options: - | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. + | --master MASTER_URL spark://host:port, mesos://host:port, yarn, + | nomad, nomad:http://host:post, or local. | --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or | on one of the worker machines inside the cluster ("cluster") | (Default: client). @@ -558,9 +602,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --verbose, -v Print additional debug output. | --version, Print the version of current Spark. | - | Spark standalone with cluster deploy mode only: - | --driver-cores NUM Cores for driver (Default: 1). - | | Spark standalone or Mesos with cluster deploy mode only: | --supervise If given, restarts the driver on failure. | --kill SUBMISSION_ID If given, kills the driver specified. @@ -569,13 +610,13 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | Spark standalone and Mesos only: | --total-executor-cores NUM Total cores for all executors. | - | Spark standalone and YARN only: + | Spark standalone, YARN and Nomad only: | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, | or all available cores on the worker in standalone mode) - | - | YARN-only: | --driver-cores NUM Number of cores used by the driver, only in cluster mode | (Default: 1). + | + | YARN only: | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --num-executors NUM Number of executors to launch (Default: 2). | If dynamic allocation is enabled, the initial number of @@ -589,7 +630,22 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | the node running the Application Master via the Secure | Distributed Cache, for renewing the login tickets and the | delegation tokens periodically. - """.stripMargin + | + | Nomad only: + | --distribution URL Location of the Spark distribution to use in the cluster + | --docker-image IMAGE The docker image to use in the cluster + | --executor-cpu Mhz CPU share per executor + | --nomad-dry-run If given, performs a dry-run construction of the Nomad job + | and outputs it as JSON, instead of running the application. + | --nomad-template PATH Path to a Nomad JSON jobspec to use a template for the Nomad + | job that Spark creates. + | + | Nomad with cluster deploy mode only: + | --driver-cpu MHz CPU share for the driver + | --monitor-until CONDITION Tells spark-submit to monitor the job until the condition + | (submitted, scheduled, or complete) is reached. + | (Defaults: "submitted"). + |""".stripMargin ) if (SparkSubmit.isSqlShell(mainClass)) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index dc82bb7704727..3929adc4ddcd0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -23,7 +23,6 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Future -import scala.concurrent.duration.Duration import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} import org.apache.spark.internal.Logging @@ -93,9 +92,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp @GuardedBy("CoarseGrainedSchedulerBackend.this") protected var localityAwareTasks = 0 - // The num of current max ExecutorId used to re-register appMaster - @volatile protected var currentExecutorIdCounter = 0 - class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) extends ThreadSafeRpcEndpoint with Logging { @@ -152,6 +148,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } } + protected def synchronizedOnNewExecutorId(executorId: String): Unit = {} + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls) => @@ -184,9 +182,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { executorDataMap.put(executorId, data) - if (currentExecutorIdCounter < executorId.toInt) { - currentExecutorIdCounter = executorId.toInt - } + synchronizedOnNewExecutorId(executorId) if (numPendingExecutors > 0) { numPendingExecutors -= 1 logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") @@ -629,8 +625,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val killResponse = adjustTotalExecutors.flatMap(killExecutors)(ThreadUtils.sameThread) - killResponse.flatMap(killSuccessful => - Future.successful (if (killSuccessful) executorsToKill else Seq.empty[String]) + killResponse.map(killSuccessful => + if (killSuccessful) executorsToKill else Seq.empty[String] )(ThreadUtils.sameThread) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala index 76af33c1a18db..61305a0770d9b 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -49,7 +49,7 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ } mapper.registerModule(com.fasterxml.jackson.module.scala.DefaultScalaModule) mapper.enable(SerializationFeature.INDENT_OUTPUT) - mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL) + mapper.setSerializationInclusion(JsonInclude.Include.NON_ABSENT) mapper.setDateFormat(JacksonMessageWriter.makeISODateFormat) override def isWriteable( diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index a72307a28ad7a..ea93cf642672e 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -80,7 +80,7 @@ NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads BASE_DIR=$(pwd) MVN="build/mvn --force" -PUBLISH_PROFILES="-Pmesos -Pyarn -Phive -Phive-thriftserver" +PUBLISH_PROFILES="-Pmesos -Pnomad -Pyarn -Phive -Phive-thriftserver" PUBLISH_PROFILES="$PUBLISH_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl" rm -rf spark @@ -235,10 +235,10 @@ if [[ "$1" == "package" ]]; then # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds # share the same Zinc server. - FLAGS="-Psparkr -Phive -Phive-thriftserver -Pyarn -Pmesos" + FLAGS="-Psparkr -Phive -Phive-thriftserver -Pyarn -Pmesos -Pnomad" make_binary_release "hadoop2.6" "-Phadoop-2.6 $FLAGS" "3035" "withr" & make_binary_release "hadoop2.7" "-Phadoop-2.7 $FLAGS" "3036" "withpip" & - make_binary_release "without-hadoop" "-Psparkr -Phadoop-provided -Pyarn -Pmesos" "3038" & + make_binary_release "without-hadoop" "-Psparkr -Phadoop-provided -Pyarn -Pmesos -Pnomad" "3038" & wait rm -rf spark-$SPARK_VERSION-bin-*/ diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 9287bd47cf113..635ae9d1ea001 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -2,6 +2,7 @@ JavaEWAH-0.3.2.jar RoaringBitmap-0.5.11.jar ST4-4.0.4.jar activation-1.1.1.jar +annotations-3.0.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.3.jar @@ -17,6 +18,7 @@ avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar +bcpkix-jdk15on-1.56.jar bcprov-jdk15on-1.51.jar bonecp-0.8.0.RELEASE.jar breeze-macros_2.11-0.13.1.jar @@ -41,7 +43,7 @@ commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar commons-lang3-3.5.jar -commons-logging-1.1.3.jar +commons-logging-1.2.jar commons-math3-3.4.1.jar commons-net-2.2.jar commons-pool-1.5.4.jar @@ -81,14 +83,14 @@ htrace-core-3.0.4.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.6.5.jar -jackson-core-2.6.5.jar +jackson-annotations-2.7.9.jar +jackson-core-2.7.9.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.6.5.jar +jackson-databind-2.7.9.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-paranamer-2.6.5.jar -jackson-module-scala_2.11-2.6.5.jar +jackson-module-paranamer-2.7.9.jar +jackson-module-scala_2.11-2.7.9.jar jackson-xc-1.9.13.jar janino-3.0.0.jar java-xmlbuilder-1.0.jar @@ -100,6 +102,7 @@ javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar +jcip-annotations-1.0.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar jersey-client-2.22.2.jar @@ -141,6 +144,8 @@ minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.9.9.Final.jar netty-all-4.0.43.Final.jar +nomad-scala-sdk_2.11-0.0.3.jar +nomad-sdk-0.0.3.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index ab1de3d3dd8ad..f04e533bbdd1d 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -2,6 +2,7 @@ JavaEWAH-0.3.2.jar RoaringBitmap-0.5.11.jar ST4-4.0.4.jar activation-1.1.1.jar +annotations-3.0.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.3.jar @@ -17,6 +18,7 @@ avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar +bcpkix-jdk15on-1.56.jar bcprov-jdk15on-1.51.jar bonecp-0.8.0.RELEASE.jar breeze-macros_2.11-0.13.1.jar @@ -41,7 +43,7 @@ commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar commons-lang3-3.5.jar -commons-logging-1.1.3.jar +commons-logging-1.2.jar commons-math3-3.4.1.jar commons-net-2.2.jar commons-pool-1.5.4.jar @@ -81,14 +83,14 @@ htrace-core-3.1.0-incubating.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.6.5.jar -jackson-core-2.6.5.jar +jackson-annotations-2.7.9.jar +jackson-core-2.7.9.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.6.5.jar +jackson-databind-2.7.9.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-paranamer-2.6.5.jar -jackson-module-scala_2.11-2.6.5.jar +jackson-module-paranamer-2.7.9.jar +jackson-module-scala_2.11-2.7.9.jar jackson-xc-1.9.13.jar janino-3.0.0.jar java-xmlbuilder-1.0.jar @@ -100,6 +102,7 @@ javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar +jcip-annotations-1.0.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar jersey-client-2.22.2.jar @@ -142,6 +145,8 @@ minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.9.9.Final.jar netty-all-4.0.43.Final.jar +nomad-scala-sdk_2.11-0.0.3.jar +nomad-sdk-0.0.3.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar diff --git a/dev/lint-java b/dev/lint-java index c2e80538ef2a5..07e0a0422cf5b 100755 --- a/dev/lint-java +++ b/dev/lint-java @@ -20,7 +20,7 @@ SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" -ERRORS=$($SCRIPT_DIR/../build/mvn -Pkinesis-asl -Pmesos -Pyarn -Phive -Phive-thriftserver checkstyle:check | grep ERROR) +ERRORS=$($SCRIPT_DIR/../build/mvn -Pkinesis-asl -Pmesos -Pnomad -Pyarn -Phive -Phive-thriftserver checkstyle:check | grep ERROR) if test ! -z "$ERRORS"; then echo -e "Checkstyle checks failed at following occurrences:\n$ERRORS" diff --git a/dev/mima b/dev/mima index 85b09dbb1bf20..f4a43522ee563 100755 --- a/dev/mima +++ b/dev/mima @@ -24,7 +24,7 @@ set -e FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" -SPARK_PROFILES="-Pmesos -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive" +SPARK_PROFILES="-Pmesos -Pnomad -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive" TOOLS_CLASSPATH="$(build/sbt -DcopyDependencies=false "export tools/fullClasspath" | tail -n1)" OLD_DEPS_CLASSPATH="$(build/sbt -DcopyDependencies=false $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)" diff --git a/dev/scalastyle b/dev/scalastyle index f3dec833636c6..7bdb26f1515b8 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -23,6 +23,7 @@ ERRORS=$(echo -e "q\n" \ | build/sbt \ -Pkinesis-asl \ -Pmesos \ + -Pnomad \ -Pyarn \ -Phive \ -Phive-thriftserver \ diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 2906a81f61cd1..c61fa5d563602 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -29,7 +29,7 @@ export LC_ALL=C # TODO: This would be much nicer to do in SBT, once SBT supports Maven-style resolution. # NOTE: These should match those in the release publishing script -HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pyarn -Phive" +HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pnomad -Pyarn -Phive" MVN="build/mvn" HADOOP_PROFILES=( hadoop-2.6 diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index c00d0db63cd10..70b94ddc9f107 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -99,6 +99,7 @@
  • Spark Standalone
  • Mesos
  • YARN
  • +
  • Nomad
  • diff --git a/docs/building-spark.md b/docs/building-spark.md index 0f551bc66b8c9..a4d1554a7ae78 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -32,7 +32,7 @@ You can fix these problems by setting the `MAVEN_OPTS` variable as discussed bef **Note:** * If using `build/mvn` with no `MAVEN_OPTS` set, the script will automatically add the above options to the `MAVEN_OPTS` environment variable. -* The `test` phase of the Spark build will automatically add these options to `MAVEN_OPTS`, even when not using `build/mvn`. +* The `test` phase of the Spark build will automatically add these options to `MAVEN_OPTS`, even when not using `build/mvn`. ### build/mvn @@ -55,10 +55,10 @@ This will build Spark distribution along with Python pip and R packages. For mor ## Specifying the Hadoop Version and Enabling YARN -You can specify the exact version of Hadoop to compile against through the `hadoop.version` property. +You can specify the exact version of Hadoop to compile against through the `hadoop.version` property. If unset, Spark will build against Hadoop 2.6.X by default. -You can enable the `yarn` profile and optionally set the `yarn.version` property if it is different +You can enable the `yarn` profile and optionally set the `yarn.version` property if it is different from `hadoop.version`. Examples: @@ -91,6 +91,10 @@ like ZooKeeper and Hadoop itself. ./build/mvn -Pmesos -DskipTests clean package +## Building with Nomad support + + ./build/mvn -Pnomad -DskipTests clean package + ## Building for Scala 2.10 To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property: diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index a2ad958959a50..0abecc9f6d60f 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -13,7 +13,7 @@ Spark applications run as independent sets of processes on a cluster, coordinate object in your main program (called the _driver program_). Specifically, to run on a cluster, the SparkContext can connect to several types of _cluster managers_ -(either Spark's own standalone cluster manager, Mesos or YARN), which allocate resources across +(Spark's own standalone cluster manager, Mesos, YARN or Nomad), which allocate resources across applications. Once connected, Spark acquires *executors* on nodes in the cluster, which are processes that run computations and store data for your application. Next, it sends your application code (defined by JAR or Python files passed to SparkContext) to @@ -45,17 +45,18 @@ There are several useful things to note about this architecture: # Cluster Manager Types -The system currently supports three cluster managers: +The system currently supports four cluster managers: * [Standalone](spark-standalone.html) -- a simple cluster manager included with Spark that makes it easy to set up a cluster. * [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce and service applications. * [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2. +* [Nomad](running-on-nomad.html) -- a general cluster manager that integrates with Consul. * [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark) -- In addition to the above, there is experimental support for Kubernetes. Kubernetes is an open-source platform for providing container-centric infrastructure. Kubernetes support is being actively -developed in an [apache-spark-on-k8s](https://github.com/apache-spark-on-k8s/) Github organization. +developed in an [apache-spark-on-k8s](https://github.com/apache-spark-on-k8s/) Github organization. For documentation, refer to that project's README. # Submitting Applications diff --git a/docs/configuration.md b/docs/configuration.md index f777811a93f62..7018db701fdce 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1064,7 +1064,7 @@ Apart from these, the following properties are also available, and may be useful spark.executor.cores - 1 in YARN mode, all the available cores on the worker in + 1 in YARN and Nomad modes, all the available cores on the worker in standalone and Mesos coarse-grained modes. @@ -1355,10 +1355,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 + 0.8 for YARN and Nomad 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 and Nomad 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, @@ -1696,7 +1696,7 @@ Apart from these, the following properties are also available, and may be useful false Whether Spark authenticates its internal connections. See - spark.authenticate.secret if not running on YARN. + spark.authenticate.secret if not running on YARN or Nomad. @@ -1704,7 +1704,7 @@ Apart from these, the following properties are also available, and may be useful None Set the secret key used for Spark to authenticate between components. This needs to be set if - not running on YARN and authentication is enabled. + not running on YARN or Nomad and authentication is enabled. @@ -2221,6 +2221,8 @@ can be found on the pages for each mode: #### [Mesos](running-on-mesos.html#configuration) +#### [Nomad](running-on-nomad.html#configuration) + #### [Standalone Mode](spark-standalone.html#cluster-launch-scripts) # Environment Variables diff --git a/docs/img/nomad-client-mode.png b/docs/img/nomad-client-mode.png new file mode 100644 index 0000000000000000000000000000000000000000..5fc27d9441184cf82802b7db27f2437c469002f4 GIT binary patch literal 28213 zcmeFZ*ISd%7d;w!jUc^9v7rI#v0Fo#G zfarvhobaDmvHx}n4ZG(vBTs!-TTdTLcN>7pdsizPUJVyZJDb-wmhb&O^x4P*04_+4 zrz!@%GduG%fd&(;XLv~}Ruy6XN6(1vXXY9eMb(@e1iXxzdh2A8Euf&7^DDL>M7hkD zq?T1)Id4Y5x1Mq`Hm~1L-jTWVmw(Zh+%hA55|>xQ$yDwmIjQu*kbB~Iv}rZ6LOe}_ z?>4e;9qYgQGIcrtk@>^oc(p#*iJgQn&j0`K|5F~wp8d~_X86_LLtkjn!g}Q@^T-Gh3@8^`@wbxmE2?_Q2fQLe7CPiu4AeMw zp^x=W)GO^rZS%b%8ekHd-F^!OhC8UR(x3oPdi$4dFF-E!R|$06?n!rfevd|p1(|^m z95jw)Dk_L*Aq9b&H8i`PYnEIyf;*tB4}(1|DdV6f;(cD^=xymk)d5vDvaGl9E}_NA zSNwqG6@GD}4%Pdw+<9M+LRBMcq<~K?COi>hce>H7_W1bnq&^JY-EH1B>?J^dsXWO` zYDd}bvYYFLkSt7p(^^rwkmUt`607Rc5kZrS;w(P{Z9@`r*yBUwSIsa1EBom2#j97rKUE%*%BAu&6^fo8Iw(Ck z=tREVR|quEkN|m|w7O6SE)^R$Ey%Pgo>#4w`70T;mS8QM9(ziG8pqJK`tv0Me%UGy z4fF=q%iYkC6Jg-v7Gx0MKYSUT% z4lGTGZH&Z+pvx;Btz&j<)&yJz!5sp3m^v;Dpx+~El|08KMTQrZ$LUe3-X1poi6K)G zs#O}qcfyzBk2!ar&3X?~p#>}ye{W*1WI?O?J-V%*mVzW#Uf;Gge<;5x!v#Bj7#bNs z9l#T0b%clst?G47lYM;ACTG}$Jn>_4A4k=T+%JrRNu#j_HlTJ6|1qaekh?s+Obdb8 z6WfZP3ljZNEx|pk_ej2*(Ve+M_FrvD?g|^*Dxqr83*v$0o~Y;e?*hLszb~DQ1kheA zF!mvGZ#Oe1Ea>g0*Xnkx@YFb%^M`_0dlp}Hvh0|A zzHO9uJjnb0Let3jK#`1p-y^{7lZV=`WHJo1cTPZnisrdg2l|;BBB}%$%wv9-!O?`Y zfd&$U!J#5B@wUVd1lPs&i9+o}!5( zr4Cl{eU7z0>Uq{|2shc>4=KHZ1sbd|0x8K5Oq8A0Deg4n@?MEYUc&jv+8a}fz3iv$ zxOWROR5SZ3KN#(preRQ{n8Cp!c~3vFeB&yeOu{#7Ru#jXFHe{rH}m)EYJMuk=(^nS zG##CTR9bInP;y@6tawBaf>t2~{JXR>7t*!Rn;_wSGfCA`HagSy%~47K8J7#S9BciKTtQf(f_TNBp_K+bZ3<$oldm)4!2d-{^8+L{@)Rv2Ws#0-3kO! zU~Zd1`q3$uTNYbRio8*R7CdtS8Z*jFZEL{=?}k?t4nUe97A$R%V(6AQ24)>V7b#%) zW9JF4aSV;2;~Dls)@KN7LPzQg#7;Ca4NFK!jGf5^hGa0>&uiB___3;3jaNK(dy72{ z!{qkow$)R^f|cf#ilcr@F0~0sXjRph-4DN=ai7oNk@j&b6-C_5hD#nnj#%FSgmt?n^7Kp8w4cCTIGu8Y{cFKhvR6^k0PD ze-%E$3hMX{|Jx~wiK=Q^$?w@@E-1i|S7a`zG!($dhX@&Fwkz@yg5tjzFG3xV3MgGr}^)vUO?S z$Av~9N@@yjfzMnU5Alg_*nX7nhIpi}vz#RGlXcItzGzlx)drQ))vz$4H0jcghnkxzn*4TFAiE|8%`;uLv9;GJ$ z6j{ShdZe|}K^5I_WpRg#Z1>>39CcipE#t&62w?g*#Ua>umu<<>((-PHMX(|@ zs0RhF_H%5-e%Mu&I7|8r!XwhYrvL|ocEEHfmBY_o?u0X)pX^@qB;+-j-{F#7sen^hs&R(>azRuv^9WX z7E5`;mI@ja5m~Z>k0ZB^n|akfJ3PX*woMyV(5U8iZjB2Kv9}GG>{g<*sr;Fj%zmqH zOf~X#ITu6CwB`$9zhSmu&^tCEyUhb&Cp8T2t#~tpc>i1oMn^y*TpEd0_6#=-@84n( zdWSzvS5Fu@BIlS%ZHjy<*M@RNHQCW!J#)x`SJXa?yg1X{tu4V=Xt}>;3Gy~g+`kxG zAI*~VtOl>MFjDb#I8d2WX#+$3K7AJt)b-+en;|xT)~#!iizwQEj>}$(LD<3?6{4@U z7A}{1w_J9Q)AZ?1IA^xO8Zbe?d&@QrrMSXEx*=Gz&$E7~Vu-?mCQ?tNMzRW3#Y-w6 zWHZlGFs%?#%Mg?*2wtjXma5!mXFrXB@MZXo+)C9ri5+fz#}b$yzIYj{Pi!Z~>_tx< z_ENix$<%F>XYxHRN=fqfu9MXW`Vv}gnP6-}k$kX|fy2^e;lnw5H@mW=TGx;#V*}~w zf%hv0?i>7}u0RRd<+Pf0C^T&N>a?l_?*Vc-=V8z;dckPTBhW)t1-KW?gEJ(+0r^ zCw|JzkULW%lI;uN8wN6Q5kWb8HKZ?wePr(V&S$~y6b=MSQ9`0R&`D9@?~VIwlMk(d zgsZjB-d3BeYn=TK)}5xUKUdSxm&n(AbJY1q(QW0961jPEddyXOpOXBBjd1$k2ZK~0 ze2>nu^$M3#i{eT#)-?&q`7caETWbX}bja@F+h!0V+n+fAU0M8NlJ#ew&4QF9n;lkj zmB0rcKSPW2Bmvp?!xiiGIsCAl>Sp?q2Zww^Fug{h!DB<)Df~AH#9%eu@#{>pkHNy#jnTJs^uicJJHTccnNC!{EV1x;)G8XWADt&+Ij*F@Lsb1{B|B2}Dz~ ze~Mz=W!xN7+GgjkmbEhp=GLgc1tCr*c4s$#uCJ*TH1zS$Q`*wJ!I7 z8~AaRLEm&Ii4TlguUA-<(*9!+UIzmf;jbujU;ihpvs6@sCOytDmi)8!-Q~jXv4fC3 z|G)}x|6FtJb_YbN{%fH0*9`%!UpMsB1UUcy_K~R? z9a!b8Cg2Qwe2Q9FQwPZYwAa2VP3TdAGpTd$7Bz~GiMTPRwDe;J!N%K2O-t!Xn7NVw z4FMtlAJcF53IjxhE9kBSt=AdnC0ssyoBdzPF_TWbq}lAU^2U=xZMSY5eB8iXCV2k* zv*3?;-urqLe4cSRDLpj|uKf^tVp%b)I8ZzLzPYSocITUE zU{s4y*vMW;m+znP9491!khx_yldG0V7eaHQ_{4TR z90bWaMM1YDwTdo*l3JT(C!B=fo;{7je0e@tRzDA}bDkb8-^cU|I?Z9WD$9mKP$GNl zoU{P@eNm%ZCqOf2hMqe;)Z z?B+ep6?O%|@Nti;iMU0{g0`a{V!c|VVvSzem=yb1aznU}4I_fzrq{Pp&nV{DzD{DHe9F=>(Z(fH>fil9-=vrx z43Z{xRuJt5Hfe=*+O}AshOds5P-{s!E9I_!$xCl|1?t7QyT2D6ficxM~GzJ3Ykl>VAC!|o`pL5}yR>G$J9ZnZ{ zCT#kb_+w{gr%tMX1GT6BJcVr3EuQ~Q(pA4IMdM15ueRi@E9R^5+Xy<>?dfYbVSO@(mHm-< z!8vcoBWB;$HYrvqxf*xQ@9=?5sAd+nnAc&XKkc57cfoSi#UD7`z=eP^;eL0Ylvf>gf`~< zc2RH$K$Z~cXTGqNTaZ{Ll(b1aIeFb{zF-ZX6@7eK@TmGukUw|;zt`~IzPLF6U3%V@ z{>3kgv+XzV$waV?Nde&`#071{o@|?@OD~=0c_qLyY#g`LXFk6HH6L8u_m|!M;P*uU zXfi0sQ8!L8et_AK8+a~h&1@wO-2UiV^nz5E(_LrcIpg11pG|*Fj-AvjLuOj`N3|-A zKfxCzA`hP)>5uM+7TR!2L!Hn z{{Hs6)you-c#p2f`YF-po|?+~2B&)c$FmnJ{@rBO94TEXrd@ zC(|(F^3dS+No!0&NKp7}L7tXYB<%UR_O8|m0<+NWreqZ`luVtCKG~RW{tb?DIVD%I z(BJ64OjW>O1pnhtouDzltPWaeEk8Kd^o)4|F9u6Z@mn}#ouj;Ofk`Z9D)Mk`D<7l^ zXR_-|&9`OI3J}z<-GZ~k1hkIHi03cTrK(@6J;(2U8;H^lMZKYm)Zgve_SUC|ci)3p zzwtUuSWT%okJKo2l}SJYM{Mzf>ns4BF*kMKCxeFuWZEiPj15<~yb1P%!!C8^&jw}3 z*dS^rYOf{kriw6r#+Jy7ioEQi?SrmI&&h~R+Jj@L-g*Yj=pxIkac)C`eVLsROC*&r z^?aHhK$Cn~o;Mr_Nk7OL0E>_(*wju;j6}#*5~t#ubRaS0YEs7t_LAFaAe_SSOBwf! zAUJz1O=j@m$yTM9yr0hQ?)_uWcMV7Dnbj)7r*jah3)Dqt?8?R6FY|r{JU*F2DTfR@ zVta9F0r1qo@y0iB)4A-FO5sTquwdSBOsVan>nRy1jAc4f1-l%6c@Pf_x6p_WdWyXo zs?NaOexE!qJp5kb-(}}X4{9+GGbq$w)#-Slrlt@jOs1kVegf-oqSm7Rk|Vl-A!Vk{ z_(P`zYB$I>kbF&qFnl}~^#X_)&$ssEn(2fNn^(6azQ&$-euVF#q1)h+G?k!i^5M3> zhYjHtmh_ar@K3*B{c8>o$V=QVV*{hhWuAYisR`VI(gmY&FHy*(Rr)ux9#fE=XUAW|ZZ%h5MXdM}FY4(&W z*I0YL;&t;yu~pNw3U-uXDb~&;QW9RbbU7R;7pa;JBDOe$q0GejSQ0Y&4zpUhBOBGr zOzVdN?PesZy?1A{o9ArnF9^qhd6U>CG2AiJq+mbtZbtpMs%dGkxtY%Vus?K=TmXI8 z^T$UxmY-fA)de;_+{~wD`F`brT|xPSsT~ZqH=tazFu3;xexWbM<1c8!+EjNXeS-;; z#u~NPBQ?o8AV8aG$uQADr@d}wl~cCKLwj<3^i#H?ZE07EZD_fGlrT_Wk+d#If(I}* z_|Y#C81_~lAp*#1(mj`i%LK|C*y9h?VQy+TX>k*C zAQ_d?Gwdpo#_~SpE1-n3Qp{iy|peF8s)z`kei4Hx`kEbRE?6gO_UgB zxMqd4v5Q$fZT0xXt}r9ibYZKnr)Tl<{A#^R`BMTlv9TErDo<3v+Q2Q=$e7}auVj95;SW*K^A12Dir;Y56M@v z|3xk)Uf^@B^i`eae%Zkv-4BVDdP<1&r)#miv_sOSA?hP_*;y{wkHxB3tpL=?I*oBB z@pW40E(pj!#oN$^QU`o}dvT0#z#wEj-rX0CU`R}rSAQVTygc*-n|c&cenaZ~{>sr` zKdD<~v97p^DG_m~B<0{@a16lbynXkJOOn=vT?*|R{OR;`C{P-t6(RLsA8n3uHwGFF6!X?IDGk?jYKuGiKaiwh@=P9VK zZwH4R#PZT_*?suBX%-ams8-+{QxdEOq~6OV9AUAGapYuYkzqJ}bm+MBV=^u)P5lW)9e}=se@!EJ=$3wJ+ohd$*HYH~ zK#Jig;w6K`RM3elq^+Yazn#mhQ#J8)PW^g)0DxP80HGV5=DlQ#U4f{q;*j75yVTCb z__GCZG7^gXCoSC)qj(BWC?Oq++>ev=R!R zB1dAMI7fV-LT`Z9?SMU0+c&2t$C8Q?@QE=RPh@(JertcQpEp%#HQtjF^u9JSs`CQK z(fQ}8#C49XrvzY0C#epWn#QlQhOz3Lf7K*VgCg@z`IdO|Z-D=96@*w!zu?_rIzfH^ z!uMxs_xekp{}5Fe)3ldt)w{3u`^UAH5T}BvD5-Qv^D53oWyY z&uum^9?p92iO#)i9A>QMm-Sy~a{vB81R(HN-zibO|H+PJKoTdF*G4A5S0$bj# zt~aoMnZy^J#2eWM=|U9`Y+3;=lNb6uuIQ}^G)tJ_M)Uqc0OyX9!{4WPo2>%`LmQ#q zPES3egBDRloN(ay!K80{G7Y)c`QEbv;vxXP!!+&Ygv+DkvQfiAxK+Z`)glIphiTz! z|FU?>dCd`Db9c&bjxK5KPl*q{Hc3y2qeF@e0bco9v&097sFqt_d- zsSqmNnjg)Y?lG{~bwiw)nOXQC3WI-5Y&kx@kB@syMEj5P+h8g#Im<9@zM*X)VWAuX z)>NKQG9-J~JW1B7mta2X__p9G*O{>fg`pVD`Yqd*d`>JV}#V%xWX z1x47RA(*h?00d71vV*w1rz{`Iy<}EaS`i09fgx{6af%o=>hL8TywH-EjX-Zj0Yc+_ zE)8{E-FblcK&uG>X;O(6$}8yyI+~o+4|Kk>0mFG{+DZPAl%8JcR0e7)aq0;}HI#v) zbxtouyTHNd9qx`doGM{m_J}!gie~+hu+>g6KR}xeo z&B^b~E>iqsK0{w>Vnd3riinNsz&{$>A+4T&l*a>4z68v{L*SbeC9DKh!cZ)MYAKvH z^Ye{hy|D{QI?J<`_C{hDL{QU8ZQ$(}9)$bO@4>CcPgzWa^OYT!;f0=9OZk(6G$F-h zNYNHEOr=J@k$<0WCYr=a$?MalySYYn$kkjR?Ys0jGf#?XXhR$Mw*G1&=1aKZ?9nxe zqZ4toFZJ;w_QSEfzP|M1sxtQ?*`?okS3j>c7(d?m4WAVZ7eF(MPQGW5=Eu-e0_Kx} zVP9~qbdhCkt#v-d2gUuS7ekGsmooT{-fs$wiO}s?( zD9xGd3{4cQ_x_HF671{}o^PCdTF`wq@lkuvW9~D^Ryk{aPr;dTPzirr@^#7iqJ(j9 z0UQ0im;2&90Ra>@J^!=TN^^hChPAJ?*FV4Bjzp=c?#Z2ET8b>HfIM1aLcG+H%xd;BZnbX6Fnz62UU(yu0rbuqs6Mfb7|> zVL(8iygqt_RKf1fm-!FPAy;YAOnitjS*AKGM9ZN?!)1QZ046~5ZosKw<5EfVi9v%l zr1Lxg#CX#7Wj85LEULE^W4bu*u{RZ&M;&<0-9^G?%}`TKc!3{N%QLX=+c;+u=)zsx z6~qci<%K{GpfdN4d+|8j&hG~|=50@R_H`eULg|0sL|(0cSvOklMswS$(>@uv!myir zB}_j=U6#vkmOAWh^91`8&kHa1))9g5OvLkg!?jNHSXl8JiC zN_l)-x1JTy*|^tuUqr`{GOqRNY*Wg9a4eT5@|vmA6;6i*F|8Tcj9cp?^JB9W3Pxn_ zMqnNR4JE9%Q9oA8rG3r1v|V`LZ4(n< z7b{e`;c}vi9DzUIvsoK;4f!*7(z-9@7avZ9ednc@)ES1QTg|T{Dx*37;V?EwDoThP zO|l*XcRvXc(*fq6hRrM1Ea|DNQZ*}pNvvvg7K((u8)jQtG4mCAh6(}8^Cshnb;uhI zX?c%l?o}ytXHESXn`!Vzrzie~#&IDAPb4TUV~%%=xc7%Z63#_DqtlLwJ)}uWJWO7= z-@{>ob7($DH8*7xmd@|7*MC9;nAG&;sn>z$85+hNG6TZzU9R30-nr<{;<@90gy(lp zxU4oqejbV5Z(7*R)yzfd*1!rpzze747&-X?i80&}JVZ#VEm~>8P~4+WK`SK{ezhvH zhVaf%@&QNK@G{{>L8L^yDLu=Fsu(>s0%MF%bW_=)@K`gLYjJ~ieyTHZQ67hTDh|&O ziPaLb=Eqv@enM!UL0@6R`w$g&A$OQ2c_xj`MV)JEQyP97435#}9Ue&_ct#`g?q3f+ z;Ni*A{-~?2cb5utCj)a=pKz)}VNr2xF*ojfkK)~D&a;a<$0t=LgN7mvqg*9RxjD-# zpX>|?;*$7Ri{p(8f#4{b2)xgl*IrO3@({n*vJjtlw4DdzpUC3>(#LzgyZy}gdEeb0_UVwo2XfLR_0gZrtU--o7ly1(?x%>~8|G?9QZhx=yOAN3L=;=1ig-q%JDXFK(r<+~ZC#|yEq9BA zj1cIE$ji$saa|wSHi9~Q7Fk;C^OA>>vE##wq4{H{^0ssyd4Q?pqQ-}fZ84~xq5Tbm z%ePO1#z&5x8kba6J-&r|=UQvu_=EOcdnwl9Y}v(2WbWhl{ayF!Aa(U%_d0*-pb+Rb zoF=(kPuHp?(9Rw5x<0ZyejaN|xieexlpt{6Rtqs36e#XlGOf_h8{uBQ1cngOSsoVu z#ro~5Za?g>K_Z(#v6(vAq#NEM@HiUGR$_a5wSb*4OZHE)dmHNJhQs=wwWP-ii-V|x zsr_bZVBA>xkz>SFF(=EGU}Q0|V+h&Ea@*sc-2!BnioyQ!83|wo+|CFuFSeIjJ?J)G z{`E;KxQsoS;>Ioey@qXJpFO5m&4aMAaN*z z@l!8dc~ozkbxFxg?7}9o^~iInBG9gg^X=uBT|q;PDfiW~=2*#dtN#jZZ*a{a9$jVn z3@kDe!r*+QM6I-<_~hVg$1?<{xHBZfd2L#nv15oQ%a+Y?rBEgBa5t&WwsjH@;0HV#vIlD_wth`4d@AKiVx`Y$|57!?--ZP?80@9$Qwr+QN zw3<;?CTF5Tz;#s9ky!u&y*du|CIoBvEOr`!j0QjcDB11M>O5wIO&RVG|0wo3P5mtv zq2>88JZnCru4Hi+#^tc}7yAya2U2Yf{`;SCWmVbOe5KTmgKFvv_yXtLQ?ck%O>OOC z0#fKj%~TpNYxu$g&N#tyCb}$mjYNY%`N(6wbar68hVhB%lHkUJfhm19A++At zQw8V6>e(F$k+MRcc-npFaGIUAgO-x@U8F<=yodlEWi_kg9aY(0*@&2;ETjyk$NW6o z4m@IbWqPk<>~#$Eeo|A0h9WPBGi?=bP_$rXWERQH4WHimQbl0icIHjCiOr^FEa?fW zw&6Y5c33AHw3W0sa)v89$D%&5AKldx4#S5U7bOab%N{({)@5K*HOl z$&~GG_$}$NW}!c6CV~N{M1851zFhAEK|L875i8j3AL+dLc_7u1${D5;2IztL zb}Md{fHD}yGiX8JeeH3 zSnEyYdyPfqLc_CQw2-cMO__>t*Cr&>!Bd()(M6*htfji9sb>4RXliD4Wa~x!Vq-`- z(mHjs1@O6qXrGX)%z3!nI`|0my?@nx4ZH2%+^?t%(kC%LO3Aj2z90QDKR2|nG55dHyYdr?7Pmq{+Nh>5s9p-kLApW!6A|6@pa9t(r1JcV zsb`xjL?Vu=)izu1gwbAqJ2W)Kgh;m&T0Q`2p$>NZU%8ynIiZ+JK>o_@cNdBNbs)iY z&|>VDzbs++erGnFQ5hHIb$4TEC3qrpazpCh%m!s`J|FAqa zIVHAKVaOiYYX5M@%dP!Se{$O0!J9s<%uBBS^bOgwjmu%b1_Ygu8L+Eus-TEYBIveQxCLfWzsGaPwlRH%*jN|AOcP7B1w9UE5lL z)*NnTzb6{JU8O`Kg2&k$k7c+@iNLSiBalopZQ?F(dmYX4Lkrg1f^ZiuEC&R|gRA-{ zjvjMD3Po7H8_AuBT{?Nk1>fd5ZQPcOc(%Fu*;Uo^tRcBuwaH{YBK2Vhn=4-YY+Ft0 z*yI0R`t4^1T}Ylatj=@U4!#Y5_1;3Rw&9h&i~pLd_SEP_GLy&gy)0!t=&i9P6G=40 zp9>IntD$Va)iu8Q_E5AI>kjS>nEciZ5=4-(LVy>MPu_HFeX5tvC`#1zLGseg%ROMP z@O+8L(po3DFhQYiyj6G0>%Y#G5LZJk=iXb ztTFO@W2|I5N)^v7*BfFR0`Ec=uY0*drq?G+P4~;s1o3iGez2LD1XdwY{UO<3Ehm_i z!7e-&g)Ryy044l+P0TrwzpWpqNM9-??wOeV}5#sAb%$#16~zg--^q*q|!xtB!qPBdU6`6)aV zx1*MzlohTgw***4B=a;q58>X0@#Z7)J$}+eH9g?q zD@J0`&Ab<{ivqqM&xiQH9)z!6^ev|C);jW->W5%jP`YP&X9k$s=;Z{Z1sCK%=C0}@ zW67H|LM>#EIEOEC^bPd0+o}?`5-|&4B=; zbk?WcGMvT*Gu&w4Q5b=!2|elfL(%L6e*3j_DhhKxZeBQ?^Er38pgkuvS9ztxVi4-# z666z2W0hhqI{gD_-t`PQ|2N#BK{M6AG=9wM;XD5EM-3RmJ|z68hjEU8{&AZ12}5Nb z{525%-@px@!T`K1N&()dyb)2lAo9W4{0xWVxN;jEcbceu-}L4%OCPxn>q1M_Ew*tB zfk;J~PhKC-q)V>t0yzhAqG(KE*{XK{*$z?wmOQHH``%CbgNyWXGzkjrifkAAhCR>b zvX6$ymRA=i+qqPc1W^;^`)oj=UM>`xlWpLdYJJes`Gu#ds699K-PUlQbCjTMj8m#J z$tY$|&fw8l{z1Wd5ZOznp+MhJOtAN;1Z!!a{e;KT_J}Og-ss^(m%(L=KYmq(0W znTXjRvYEY|Ck{BlY3#Q}0# zAF@RZkh5VEs+g6xQEYappXK+SwXaV)y}(3Nq@LhJv~V|DPu7udQubx0$yo28Sk*hzpo+++6$CGkPq>maCj?er!!(R)r7HSsWrpk;7X()j(m5)t{x@5(7} zZ8(#mu?0P=bfm+reP_ekS-4haQptO7keR}ZQqQ2{bgw`Ox#rR%hE$UG(5~$KSO6(`hy8ifMNfK9X}l(czU^vQ3q*@moMt1?>g8|!MpIv*iBn+~5Vb^z zizD1FTy!-${L)FU=Ouzaaz2Y;yb4NXFZ?|){E1nZj+)Vg$p(zS_OcLKL7mlRE_#bV zCxeoP$8Q&L{Pka28AR`G8=EVn5we+$ch^F;ImddD3=(Xp4nI&e7xEu&AamIr|@~P3uACndbX?(BJ#H>!XAHY4Jmea<}w5_A+@)L9R;h zfe4iDFH@hs3t;(D&Nf)btQqpHE>Zxpj4<6_&{`UT>7;_WRLIm?>-R=h#(vqfOlH6w z8pb1wb8wj#XM^MWlmPoPQ6meU@`?=Nc`ZIMiJHd5F8fcN42ed`R<#Q2xhgWHl&XI+ zex|^ zaVjzXTJ1I`4WyXbG0YzxlB$|>3E&hyHB%zB@lRmoX&MCeUtM&?^+@|>C7n&0&?}-j zz9ue{#!W^ed7XDWAjmr>>jCp^hIK@v2#GBeywV9sgOUyz1cbi4c`Zh4^v1Md9b8``O2KJ;$eqAqnp!&@ETE=s?8NJHbI+pXN_yhm~%U z5B60ioQjkSTx?r$W(sD=^hT>dVy-pVQc*Se0!F{Mo{GGaTBnO{PHvfe71qrEYebDe z67thJwyFSSWhZNbYhPlct97ib(u_MkkJ6}V89f{@aLufBp3bg*d8vc{t84xpS&&j} z75u`%=WVBIn6H0J9T?pvUHAQa4+CRu2exFnfQ^uW0suAlu0@7!?)05&^_G)csPrqzFs?C3yLju?sCMYRy5eilaiSMs!$|5lYNuNQN z#ySNSQtZ3yw;x|1wSiuu$^W=5>$~`gPsF`kW5D2Tx;2V?utg53{x^5K_>>?h35uA2 zKywH?wNbZAP(J|IbXn`2gdhRPa+B8@L>5PEakkV&guiJ(xq<0#NZuT*TJ$30LAjpy zJcV0kWoUP|xG+o%xR_qeE~ccWGR)TFAAs(MC>STrN;4UI_Ib6V(1%)m(&1Ne&6oN1#oU5|=dfbl@ z;}hT{*;&|qzTrDUW^@E(NC@T`0udS7R_X|iANS}XN#vFqNh)YgoE&9kPXE-xwT~b9 zZz5pRnDwI(DLscTsH6@t)2Q%$H9l^5#ae4D|43Z2w*4LYCPl60T|RokULj!SWb#hI za8VIcA8&Pfs&Z_IRrB5ipu_Mi~tx6C@L2tzif&seX-GROLJb$LRoUa%UO5ZNO#Vb&G~qSESvpa{X`2}xH2MeOCqkzjCTLyWa!I0BI# zLGY<^0Hi&MfQkr}a+jAfhfl?Qf79!mnGJm+8U}S)Kp|%7-NXIQBP=FBc01#GhG@>{{JfU{=LU@;asE^wmAMp}vE3f$$Gy!;GlyuJZt30{Mvx}QGgxYPU# z8~|kO%T5v-p#}UrJlZ#xg0-He`*lJ(YMqdi0lQUc;)6e97#lSJO>f!8>9v5&6WW=< zl(ywUmu3vk%i z@fQV>P_#UKf)vK<}~KKP|L18=yh!UTWq5mTgntbNUq*^40G1=Y&_{qODo9W29IK#(*3uipo>qsNv* zL0T9YS-$;={h3U!B1=4*rTP>QAZN@bBo))YSm|wI!7x$tuAseycSFGal>40p1E}kl znu-&iFrg6VVdDp_ z&;oE-_RFrP!a{bPY^O>73?#4!6`svpgG5RLU8ofBk!hoQUWRXePEIG88dlr%%qF4q-eJji<{%7$DEG{GU z)p%qlZ_vL(X_ud!PYo4_ntA(pdf-1kfpF^RBo)wzA$!n&CrhI5jBnlK$d|Jp=55nZ zJ_ANPQ(=>z^c-+bH;w810U0HTZQqFgJW=Jmp%so~9yw3%PQjWnq0!sj4XX6>NQv^*uxl`eGZK)s&+ zH1VMAQxG~oof9{D;aB-U1o;P|Xp_>Kt%mOK@ikLgd%QU5NdN7Jspa|fJv z6^;zaJKUc2H!_K?rx{&g)47tSPk~K+;B%?#8r$!5Iv>R=Q&#`MGoecXYyCvj#s>@g zR7W|r@}SOcn3q~w4zz3zoSqH6!w~cBz?4W%f__Mx;*uO8-q=h~n;l~z!BjM4+q2=p zYM=v0RP%BX@bp9f0hxX;`j-5GdLpYV@}8~qJXj)4ckY&{VeLL*(E}#F*5u!>#n5 z=HRoG>Z%P+14IrazpDFs+lAI0p15X76@;KqYG5|{HYbTFdl|=ueztO9TO+Gn8DIIk zI`%>|WIg@7${fWXmE$@IGDmVrgF_ZIl!b|2t(Ur6f-B`CODNv9Cql0teidszCq_V* z+*`7N!|g(s%S3?otCm{44;}$+Z9xcf^OdSzw#6D z=bI&RUrEC4Q?JNPc_uoB7;Y>dzJMUF^j=+W*bVQnSGNT@s4bV=-p~^y=eKU&_&k_) zl~1PS%v2x`A<2c?DqR4zoB_Wz+V%GrXGsvQpGOEWDq|U<@~0(Swr9S3?I7Og6f*lR?gOrvIP{4g?)T;1ICgb4gm zR_}vzP5;IN_#sIBf2Qnh(k&pCxz}cf)x#?9m>SWN)%?{9Jx)Y549omgekNMxspb4W z{VG37%lWB})lo*FWeo5idl)K3^+{1Q`6t!KsC?XtbX)&M_`4^MD!=&a^p2Zb-ECCA zOPI<(ZnFk#@iCn?5k*Q=HYM@>s@H|0x?8=WDRq^BbE%yT=ps}}^p6c4hCoN6dwTic z4oUZ=UK2){5FGPKA`$hIrm}ZD>k4vT>VNM(%{&y9vOCI%H?q1rzassri-GV<2-gB4 z7jCwU^eeuWZuQna*YxUc2=RMC;mR*NrsK17xB}Ei^J?s)<1^ou&jd$ZF9`db=kefL z9t!TPc)Ynh_&`avVI;_I==&z7N7>%FwBLx0+I5D*)aq{OeS!~Cuhu>iv|GTC865{c z_#tv=Mf#c|4Bd0-J=t87#vgxcwA=jp5wnr9huLdQQU1@F#CrT5#5X_b@x>?D<0e2F zVX=abIFqOTv8~z6g(*&}FdpeF1 z>}JFeD*L`Qjk1n0_MN%U_*~!Xx*qraAKd*gj~)->ea`!w*YZ5i*YkBuO23kH>J=$% z{I6+?O>O;&k?T@gjZ^?ypm&WPy`S%8yG$t3u-qznwM&kwU=KN~?(@{@6mXZ}Gf=T= z4LG^l%3a5om>a_?M{IXDt!+l8?Fo)uk$wHu0gQWS7PU7(aRAh+G|5VTC8YG_J+pUEq--_>rF_FBS>G-+m_n!|{kZ|N0p z-N}OYbja07Z2g$}(E-oa@!rLFA?^I#0xF6M_| zY?u>zUqA16C?@S#{9xYIOXS?RZt{1>R2w>29MsjKH*I3`VgXMwYn`lP%&Fx z`Lu7_<(kMw^>CjINaW0Wb4vXA4bbFtdf|wgKy6tnUisOEJEG6c{VKIf!PulVsBM>u&I^wNKYjTSt1VW&p zl%3$xCU2i61d`sXFS@$1LeO#*T1WuOcC$5mC__$jl}YEDh6Xcg;}s5?W_MP8 zC+ndSO3IbX@Y!pedU}UiCTrvsM|xZI$#t>hbB8&w{Yx`aMRo!h#%}SwJYiwswV4`O zE{j7jKQT zeCeBe(hAxGC`Iz>9s@iOh)WV?x9cVGc3bJ^{?og*RWiTnIrQcC+>rS{%p33-3j>_c zV`BMCbYKDHP5sGM@pWQ1Zyz=GDu7~`cSA26ioFd`tsB+#)fU~+JTkfcNVPaqr9(Ud z$+YC&@9zDuk#K({LWZ`1$#Vf3rYdJq&R3_^zVToe++ z;NvFu*(ccNj&*TUqbSwGxAcI?hM2=qz0$>n&0TC%iIxl4wnk` zj{cayeFk(H+@B2Ci0MBlRdGwTS0Ab=((OXYLRT*C2P8P9-KFr2mE$%Q1sTKMHtxsQ zUS;L4o?zJyyi(iTJh}g57WTQda;p=|?yCfZ&o(O!Ft>**mu1ImYxA)W;?hh$=rlYH zbe<|*WDb}BHEqVQHh~u}UQBnZ?|NP8o^ssEl{MnyfFKCy%5TW(BM#-=vv`|)u~K5d zx%J%OQ};8suwnVy#g~+oOMe%z%WWu6mtgEH19c|unk41(GwJGOiF&R5-d>KPKH>@v zXdU0mW;KtEICZ|HABA~7g*)j+73YVIv7Ri%E+-i+egwT)z2Mm2v{z3S<;_S|`4ge# z%z0C3BKXtKGA)bbMS8Sw(yb3&3@uk}b&D zS|Ou>DgGU%I;QOH1-c#cLJBi(P6W7@QI^c?)5xi-M2nZrx)`%Dfai>}Hpaf{av#ps!|>gKGQ}C;3XWRm z(mTearHuyD&J=W?^qmOAm@{$u)%P4}3r(yN430=S?o;%2xc{%5hKZ$j$MLEO&(^c( zoQj6mdiPv?eL)!~r-AKeRiOrFk5sT(HT_U-ecP+DB>ll}R;{!z=sAge~mC&RvO z1O~5j?)Qs3BZi_ot33?4TApHDlAFGgL$vy)&hn$BWagCY8ol0B|Uf@eI zO5eaDVSX{|jizWZ1#}1TXqDVd`8-W?_9R32(kmKAtNI`v47TIK7&(eE7+uqGnn%_i z)O-Bq4)_}fY!_n_JfhsS|Dx?>)?3|;vD{zk>zv;##_D_4*__UtgEnzwuW0gC3Y}k4 z9*5}&i-U!ow|c2#E=XZed<%u=~&&!%(gs zcf{tc%aW;SX}*^=e5wocJNBlCC(n z6|O>PHMaM5rIDCXZlrsm_Qx^T#?I{*V-=S*z1?1!**<(Far)hzQ4&v*zlydNYK9hU zRqfH}ta(_ySmJv@1A@TI#aqnSne$SY`kWV++hRU!6+dZ9@z~Rcrr$S@6(;22>V!eJ zwLlgIQu}G=zuJ?6Bazl%mCd;=>wN3;3j18pW0zWa;~ArpTYOa0R^JsP3MiT^I=)&@ zL;cTG)Q2Ks^f&pGR`LyBX$QCk1pW#S%Oau+O#7Cs7B#klXlO(=!KAYCCr8w^`^G0L zl#~4-6+qo(Cz%EIUJ$LP^&r%4MtjxYv9eXDElE@Gr>DBOuDq8R) zk>r>1o%iS=dTP6cqbwD&K+Oqg4DuQRrQugcV8HA6rPx=|FLEE^dDZA;1d7`fgTn*4 z^7RTzNw=FfIol&Q_EFoD|2$+K)?cD;#P(ybP>T#R%<^zzgUrEzx&nZvHW~hd`2hca zFqEYp6o1hW{5jyudf)dr@({NcOd^BQM|75#E+2GcC#?3RkAU0 z7sCHy%X$bd|GX`>e;tyOe49XI29V?571WfE9Cfu|gPc7Rhd^QY?kD5ak2i6?z9O3K z#rf~!YJ=npO2*2x!PW`t+#iBJa+v+kTh+WAPN(RRcy@5=VQT+9yb4b?85X#hWN%Ke z{!)K)1YYeop%^@Ds7?UqADWdt2*ZClT+MmwTen`36#6T@284qX=>-s|e?=Y&91uMC zg2ns}gmWU$8w7vMNI95aV|TCG;Zw+%6IJQEqu79|Yi4HOqUgH$j+EX;OU&|rG?Vc7 zZ3s}U!G(TV(tQ3zn#|vR`}^LOze)_TSnFWdnl z6W1O!9iWXS84zdl633ft>Vo?(3!{Ei$AQL^o5D14I^>`8yAUId&lGS^1x?c) z5Q=|ig;hU(yXqHOjrO2tL^V#Pm{Ou1i8nqWfy6clyA79UaZ|5h;l(se2p)Ec`|wt$PrN5&0&&_-yVS$lHLJx8!f_qEt(Ii-LD7b`X>n0HF<7BIFejP^~_&UZ{zxA*CVv4b+SXCWSBW1 zY)P>GWPpY*4&4^FSZU6l+l-7Rk5LQMBJTbCt<_*n>h3mY>3MR#xadX2%G}T$!2d0v z1*5~g$G};WF8xoFbtVD@Iuq7-Gqe?MS}-;W8bL5~m`=)6)Geg(gN9wc zf^e}@gDslmeednr*(??6L1(K)p3Bo<1pmfxVY1l?+EJaCANcR>qR_@vKj^cDz2pha zk>Uooszyh@sB5vq-Jp2Vj^#8;plsDPMIsi_zwMzTP|~8=%^)-hTJO*vD!9X7bbj2v zYQisnY6D!4K&Lvx{Ey=<1YW-@hCt0St*z#=ZH;jtUGM1OVyMQw|6 z>uKnGa5-`MM%=6%PdnM=J#t*EpKW|Nb$(aovRnl2meKM2h03(C!XQcgB1JMRN{OMgs z1kQb_@wpCJmZed&VUjZc(?wfVR{#Q;qb*_UudMEbp7_zr_Jd@;2U*XYX3x9!b-WX#Bt<&blx zi;X){YC%B9e*yV~)_(`WiT^IwB1}5`vDQrK=i`uzKUr9s(woNl&HVXTu2-;fZx#gM zwUtmZeq-2Yoo{An1{(or@2b$S9hSNuXU=g8E9$B{%6v9fcP7;MZM!GSf7@iakuL;D zI69vYk6{M_8X3pOF5>vM7hWBXQPxo;p+^}PH;yd*X?JU`=7zRBpAj;*)^6!#;ocTDr$q5 z34xNot3=Lb+j7yHaz49ZF5_p=vaC_ZJK9z>j{-$iS!i@nf$!F&Z|SAi9`cf{i@v23 z#(D1DqLQBN*aYd)=Fr6si?~^6!Jj*4Cmyr99ie;a!POHPv-;(@GiN6+Xs`w8kk+&a z|Mem#pZgpKN40e}Mwp@D<+-u_%v{5)84zL>1xcb(i+^;q3zCb?Rnw#bRFDc5DXp~t?P&z+lfMS1cn2jf9- ztXa2Az4x5t&eWgdi^pDWcr`z9_a#wdO00{AFC!z*?aoM_jxw7m$!IbM`TJ**RMgIy zqdQZ3esy{7lA{2C(g>Sw*btKOprL`hxQu=*BG6+o)GNgpC38NAN-A`1cBOiHS`dB_ z%Dd{_sDTj^mA+`R#s5{6eKOuG@sK_A*7~U%nL^YkdRllYz*}2`x~^e=hZ+X`nF*Co z{sRhn9l@PRw$ho7Z8whJhC+U9h2;l?-rDt0TUuQWA#LC%uSm1?6GkPEJ>6Gq>WXK}yuO-oJKgh{3JXkW^8ei-iZ=HF*sG|Ea z26;Al|I6Da6S2MjyPhaCi}4yhzr%9YsrZKH>d5e~`Ly?z%MmS)c?@`6ehQ7s96VSi zJMv&25=(KfROPhu`=tghyIAQTp z9k13Q*9`^wOM36{1r5J1G_z-5J>TSyQV@D{#^w%Z-7goxx634`@dryk0)3c7;if%x z&x);cwC^@21IZ@y%X`hW-n8Hb!1sSC2&J!$PqMmro(At!>yY?J8hm=Q1JP9?OM~r6 zOm|;O_@6<)@R&%ufivHM9h!LD~C4 zKJJ#NsxINeVLnsGTZc_dkLVEf1WL8b-+>pXa*_Wi?f5RAn-8&2>h>byWl-Zm1q%|x z_n^Y;2)wNCShm{k(%Cu|7CWvEjU98UjS8XR=~kqh&~&`Q_0%fj0@Sx|ZTnV(p!Ji#S=G4?0uV0aB?r!m84fycaHN$ws^CwDN zpypS&zY@HVjs9%eUXl;Zu30TLsl*WU5J3Rp9NDP(AXw^VrNwdG+{wYocRRZHj(E-H zJCJ&9Z@rwb1@)oBm|W_SSV_@S*D_KP5fZw)!CPcr@tt3Gpc_8?>&U(GH_Ah8uPHTq zBb&#xjBk!L7rCh=&?I>3Ym-?9{hK+yM|Cm?9xn0jIp`cMX$~WG{QfdAX zpssUh1e~lWklp`pz-psTu&zU9x*cpgmyH`C0_z6cs~x{7S!%CAYp;m(yC*uHWM(;Q zMEGsPa%(`ZpCWd!sVP9;E@!y~N8A1iJ3MdD zd^Rh$cD8;(c3kOqd`WK6i-F6?_YdUOcUjm;EQ9@vqq_+!aRYM0&3X@wj>%-sDuYrH z&A2YK;;m_bA3W59T&6?_+ER{_%}dr-{OTrY$bVYw|NBRZPp32sAmci;&@_q-WBQpal1XEm!XOb+4W(Ecp? zVpiR>*k$wDv3okItWLup`1K{Oy<4@q<7%)gBx$eO%FlD# z9UtEoY3`crx=(k`97VyBYrO3@em);5?%Mvup3-T#xbB#-IgFAN^^rJH`J;xxTQcn~ zIWBmmm$26otxN485b7(JLX5Q83uj_+dR*rhS}4X{)M$FTN5U60S*%55`<4#&0{8pj zfMG4~LKO9M;`Y4$;NEd5-5j)la|L^34VDKxv`Y8%^I6(t`V9^yY)<8Ujw$eP+ViBg z_xlxQ>|0win029=EDm^&)0HH9$iE-0@!APk2Z9dj6uWM}3f5kCKaA6|V92Rr1y9NG z%P@k^vWitc48?xZ3T>>*f4!T*7hymVjYdyKJ$jaLj*4npR`+1dDNAB=1 zqE*Bwt5I{gI7=7z#&4;pZDliUnmHE;@cgPESod>PZon1@bycN9)?ac}enJrTv%i-b z4h9@-6`fkt?*%K~{NPL&_K*3DBQ!K)icTNcIES;Y_FPn<(Jv%MRKy~DiMjai?8(@*2FKS2MRif_CKU-Y1rr!{tpy^x<0r_G#GsMTv@tYg`ru{u2P~t9ETS z76&OFtX`)+7N@|u0(iiCM`Bq`$_WBr?IeJl+A8AU_jg~^pQ$xD1(^XUU(`#zwh3+i z<1hK%ksxLUTDsY?4b&f&Zv8w5yW&CUMK+qB`CsPIoFF)43HcznNav)%FqAD;E^6*&^r`H_w`#SUV{Xol0TTZ0pt483n_pdqrJ4O1drHb z_*UnbyWBj)w%;RS9%3JOk@t_28(8P7T=L#eMJjfT>y3|LdA8gXc2|C^t5ACr=F@(a z=NvRehz}Cd(!^B?7TNZqN6QEg$2;x{(3PO$^I?0$>&4RZ5IAKd^WW7K=7S`FnXZF@ zI>Q~aVuGix12a!G`1d^q7*W7hH1#4M;)i=P4B_(T(j*AMsTelyN!<_eKTDfxtvHWZ z4Z%8+=fGA`Q)qlYsq6hFERt?`l;R;W#H*b4nuy*Yv3^u2GZTj+*B&oKW?!ZK8gOwj zQ~1`mnBehTSV2dJs4;vi&Db=IW=hZwKlMvz!p$*d?)W>M zrum;9_;MK9tC}XY7 zq5EB*EGG9FdFCfaXUr)1x@?;b3E=EHV(7ba;a@Hv54fc6Cx5~waA-!sqm^T&L({HQa z^0Y-~wOZ=5X10)fs>6BCn7#6lZa8S<>AlDp{n;Doec@x*Omi3eC3pLwrPY{elFrOn z;nE9{hQYv~Vhu=~`nJpul|B#LU|yiPVXd8cu}a00tWAVd1+PNDmQRPtIRIo|vUnq` zt;@(o?K;ns&>BtCPg84O+&od{-9mm?X-wy=5?Wi^a@Uufa9TbIB{!5vXE7esT zp)^J(-D`SO^S7I$nTO!b`ZmYY-z_GPn@VX%%?X-MN?hM2&&G{pD6CxLP&+>Ij-Jly z*&hX;PhA2FkDQNpPj1Xw-VC~7EFoI8_ehjK&0EJ97&#oY(9M=VPV)o8dFxT|pX`kT-e=EgEkzy0HXaz}UpYmR{ zkt*CZH8az$d{S3G)zZF!;50!S`?Ee{*Klm>Vw;yyp^J$1xYH)(nCkVpQXk6GEH_R$N2%)G1r9W96N@A}?q9ZrGL za@e7Y=6`*33;4~_)1_Z5ntvU=jQ*cdl(IM#r^|Eaj?ePZW^pLe`M*A%*#7?ccHbWj zaKM3sKYeY;z?~y6F~fymx<_m2YEi&C4hi;j5`xR%t^uj6^E4y^_24dA7Xn32G|AbsrUl)JQHARnFwz;SF$qZCkcoTUZeEMfiE)YevBOsz zL$f(2qMlIC=2vd6Ysct>-N7t&dx>wG?hb0M+XFxpyW^$lPP2(_(Tk^)ljwDhCBavG z3dD!yXdCI<2adWg^VWaH5ZR+ev-9sL_Ye=yNl*i5SX<;gboYQxy-xJ1cb8pAX(0 z#o4SY2=H4^f5Bx=*Y&1SkYt4b)<~3=8Z1tsN_|_L{cQ&Jeea1I zDzimr;d}axclfWIW<>RFvuk+&Ey>=b9iP(!@K&MqT)YdGe+*c&WIv_`3-jt`d&n2d zh0)A1At&w|!-5pn*get2AWd2$OjPQbFt zIFL49E*TF*`s4RsRCHJ{hO%bi4Zue1CAq6*;jQ{vzFoXgPC6C(Eb3)a(Qdpye(~vN zHoD)PR?n~S7JW|Vh{(SgAH7VO6LzzHtM0x)*m~&XK5eRnfikk5?J&}ibAFY@yC3jZq!ehy;E%ZYBYiSI+E`rvex5*U3!e z9QI}+-YW8`1!nSlo=+;N0c&hdov6^1xD5c!Y?cRCCer;j8!3Hh{Z@TWJ>{Yc2Fibc zlQjAgw2$nh#RGpu9*ep3O|OotO-gU5ZzbN2qHWIZo)@d%OqrZ+9a=X{FaS=MDtsqxzcf}%gI6C$jSmy+3Oq|Sj8AS0E-R2`j&6g>UyVg4E z|IuDh8G1BpcPT_ri8SDKLF7k%RP<+$=amj_k79rWepL#^C)MXzht}-3C@pd(=434X zXzb;mbxGoYTwPc5%>({BoNTq_>6FmhF*M{E#rV3IIn<>R!TD`xoyN6(6%50Z1CCyi%Bl!^~(qmstPY!d_MqQyYJzoQjR?~<}f?mbf+jKmyRupBW+1#jjHGekWGv6c92mjW^{BYg!$jfH3YuZ6 zQo```&iz27~J2e*sVtX}^ggsvAY literal 0 HcmV?d00001 diff --git a/docs/img/nomad-cluster-mode.png b/docs/img/nomad-cluster-mode.png new file mode 100644 index 0000000000000000000000000000000000000000..4ff12b384a762d4089c00374e1be815b0a6ea51b GIT binary patch literal 25001 zcmce7S5#AN&~88kLlfypM>dPJK@VuesG^ z1Jmb9asz_fnNGM7Kj#LlB1IOo%}YA*3)_gak{dfHE`(L+_i#1(_bnVEy z-LxlN;&{MaH2tTas7N8k%KzUlFfpTE91UHxkxT~g!Vm7Mdk+Bkk$$Li|IQE@PMo_B z5`3(?j}W=LcR)S~0KlX}WGMQrm_AlmI~CiGLPoIfJ7(`?Xzyv{2c&JykWWMUYh(tb zj$k`QC-#M09)X!B!9*c&%Cp2c5!uv7u&C#mJm8W#$k+3D-&||cM4H5VexESw#JhDu zdUThO(q&f<7hT%VJa=?hHsU z&F9$o%#{%T{w)13P`_|fAB&yZxu!!T=212UKt1w%Qis@Qjpb$FN3G8u4~g_Nk?Xi! zURT!#Sd+U)UmV50<3wI%J8uBoyg6{7A{j(RZko+v6CX}#TvP!mJ(XSU3mrkH;iK!M z2EOF#UyRY==;OT?pjXeEL~q7^zLX19mJ1suwKts~r$sSyj_^EQ3WR9@3QHn=>c z0jH$>0SpB^2(^(WjWWUyVJRE_k<!vkC1=EiY$NUTmJF z4X~a7H@7kd3Vvz8Tzp!VRd2e(^7lZuC#u<|H9+$la++*24$vpGfvhTjuonS30fT ztbVWTvlwcbnTYobWEg|;FD(<1;0JL6S&+}1t2-26qN-JoB%!yxTywQ3!k2}aB_Ry9 z^~W*kiBp1ejyetfFJE~>L$ra)FIZK6*RGfN?*H0&X6d`nDCHv9n&BGfMhG;iUQgVv zMSUL0ZyQ~&f%-zJO!kma6X#u8O%b4-pnmnbu=JOlVI(xFIjOQ_jD_nwTRE<5bf1^{ zVmQa%7dGQBe??c*er_5v0cz}bwmE7`*iPO)!iDYm&$aD0KDE~K7P3O8{w0;+_`}|j8nwiC=M7juQMM{`46d6i z6d|A8CEa>2|Cvtctf>b2+KXlBRcFTT8jYqqi09@m{N+Wa)6eIQdl)o7CDUqv55BZf zwm1*lvygTCE)YJfY`pCfS@`mn^?DkWyxx#3yq$#w43z^0A$whR>VJ_*)bTql{BABZ z-BA!gR@PL%Z)ZeSM~)tE{P5fZ4L?UoN;UHi)u;J3qEv5w#i%4fSK-5%=$;Q8vsiH_ zF{R3p$#Yi-Rr{rtkMNxUQE{)~P zc~B)=gunJ2OM#0VNyeqT2o#~0v42vL=0@nN?v~vnBFW$%OZT>+7ow(S?%o>IP#;KW zv+KLkU};K!c|9%WX&enr+h8zmn4_Hbvf#!lh+<<`uAefp3D*BH9;%b#=E$RDn`=n45Dp<%Mpzu85dJD&cep`GmJgC3Vr%~0-Pejf!)g%lgY{<-S{H0QyttK6{_JF1{sEVB-R#}yoA<^9!)uJ5AG+0PAFMLU z?O@8+_Q!Dw50^IBGOi?VuE+O@h8z*(aDbw?$)a6yI(j7R4BgDTo0t2B1b_4qi;%k1 zJZpg^srb~vgqWy5NWD>QP7%=}NnLbb3zj-7!&y;(WSTB2OaW)~ptDXiQy7rKFCcKX zH<$3ZjEEN`)`CfI^#JKz=#j_LPb^5)i!%_Nj0I$tbYIispT|>nkL*4q$c9zpsEb)B1>W6wHzDMac-FDeQP9ri?~n8xA`MX(>iR{hn{-&Xn<3b>`wkAJd%ijM;thzuzv=urEXvEG zzO|qBXH7rHPX-~=^MSLK-CMi;vb{E{1uafB;c5FCs7mqH`xN)8+RSc^q_Mn4PsuL6 zIPGwMev`xzI7p%*f>(W(1!9X}lvwrkf~@zr-J_& z$-K1!3&9$0-VvtsmNWlhVxj1MipNxYQT%D5YW<8L@`$h1PiIFEhn|7ZidX6euVrA? z#<-fbsh9D<;}T1)^UL)pKS;Ik{E;ktaai{xZ71EVUXKZK2|3SdvCKVLVmr#nnrmR` zvqxI-tVnuPC^qk{RaP|n)HBx?%CLg$9X}r-!_OI3Zrrkwt8xzhGww6#s=Mc!G2 z8en6!QwHNe?e=Dv)13W6zk<{-Uh1uY%k2|!A*ULO8>J=oV>}kJK>#4(5Uo>qWu9}3 zCcg~P`fN!MDkH{BPKs$iLgcdgG%5b4|M*{8lmC~OWR8B{^)C;8l*QO!7@Ycn&`{K(Ai$tpEuZh$ zbD_pTxE@=MS=;m=jyM!rsRL@42lLHimbzqa>I!{<- z^JPMsHJx+ccbb1R!pUaq$H*3_Sv-A5wuz^z?Ir~sxMDDm1)pB9{ zfDIRpi+)HaGX;L=S2JrXt2bOSN87933=@T6+_GVz9db8Ss;O^a62&aMpLpZ*4D8PM zpJ765(nYfVi=~_z^T*n+w+d~gB_+|67$?3Yq}*Ux#w~=ng!3WTNFKEHfzvDdmthj6 zvGu*LU*G%JPJU#47^0(2O=r=W2PUYH8thVu%X0X%N(s#P@b&RWtp^yBSq-U^T!W2I z#!1A@pEwV49$8 zbd*Tew_$NLX$=b?!G8s2Q{tuq2M^9)M)6 z^HH3bI|@pYXiPpF)2o}tsZ-1*f6(SYQlaw!NYG9D>7Qj_hdawMF=qXF!Y%0J(qq_K zU`M756~cm~0?TE>zhdsf(b;x>pDd4S}! zMvuOM-?sH>R}+8qAPgSE=2*sc){UoxHLG}hL8;3mABLT$ABNS^I6shM1^{?*zK`rT zJWU#*LK$uJ9#zkH2Gn-l{vqHEtS=Cf)4%w&`)jCUnQhG*G8GG-vOFSKPu%)DjY6D$ z?cArIPLnJvp13i>lXE9bmk8GS{F*4_=$MIlTMy)#w9;-g=in#!0i?<+ZazMzzh2m!fWDqF%*~WE<<6&;``2vTwCRFt9~ASegnr% zLM&HTr$#F%zgYUAy(fx8@v|zWR|>8+gPB%_B4!Oin)reegc@4$Qyuc$H9r-&Z@NBb z--pCRlsjL<1(lMI^MRxRxeW}IH~0=pJ;*H`dCR` zwb(F7NR|HjVGHvqz2>7`INpS^alQ~8&UxYqLv6}%c2v*U-~1tKOddIJW}q$bYQ(a6 zad$k;wLVZJFA^o{yd?4IZTsydpH8jRP3t7sLG{wHjvN~xwihVq?R^v;@p?n+Q8^U) z85l&mRw)wu^*lYRA0gRzV@*gHHveN|dlK@^5sR?Bt))Higo!3t_q)7leF%)b;^P&h z5i+!2Hk2XZQ}g=4letcQ2j3~jM^SaU3L63&+jAD7{?4`e2R5K;U-~Dw4j%1NF5sA% zhK(eDX-TQ)txIjcFJzCU(!pVxPXwQm%${OUgC+a$jjdNE7R4quPy1Wb7t2J}#68?q`I^0i*C@a^~X}t)46c<7LgOl%>nZ3S|qCO&#ORAk|gR&nL=pIS*<=e#whb zrySYd4a{8hy^XMY7^M7~{LL}Pyn1pEi(t2wR63ghk9j!CCZ`Cj zo%IUpHN$@#O!@a`;{LGEwpN$R50NoTbvSQ=6QCMO)Oh1M&8tR2WG~ZwwF;F)MjPUC zFSXs#?qL7dqp{-#o!FRt{9yAk74poLl@9*q znMyQ6UGb{IrgWU-x$*dLu_>k%oDE|U`E$BETA(;eA8FwK)nLE&x9_mv9Jf6QOE9J= zXf}crtvoa~xs9?r^jM@(f*#u4bp521R_t{82ez_mX*Re37b9k^)Dl?vm{ls0Rq-}C zpj3n;LKJ=QaO+bfpLUziso3gE>6Ued)4%(H{6Sj3;QVi#Y!dg9%sHe_%&hts_ENY7 zEsCCQ@GW2dZPWaK583}!_p{jo>zvVXY+;<}li+dD^+O#zUNc$5M^w(cCKVB_In~Xu zv88(+O|?l|2b^S+3b9s|MZD*f0hjXUrk14in;*7#J=n)Kq-56acb2SqYE}b-BLyW$ zd>jQJEbSZxYMbxL@`9Q$6&Jutj;$xe8}xN+mGVf|i` zQwChN?cRsxFqSs-T>QPujB8n2D&L3g4P~2`JG0>p$u(CB)l#pj!H<%sWDR-Vz%u$S zq}u#7PNL)a3V@#Wwak-<#-25u%Jx7gWKWa@CGdeHog8=7Gj4#}r?M}~HMpo>9C@HY zMr;L)WYZzY}Ih$yNgr@XN(DmC`_M>Xx$@X}JdS;%rv;~b&4pwrO# zFE}F9Y~FN-u7Y+|V;~_B@&ob2xn<6cgPFUA{ayMNqCfk~IJi2C;kG~jQ0(HN)$vMjo(}wS+7pLF@~!caJFH z*R6sy%j>39aNNb)Z#+hs6ccr?aj)NkNM5eyoHiT~%zfQ{*o`!sUq0PuL$q~DV3$zh z5~x#Or-(JDi5qdCIbXb0?50i0_@648r_MZLzumH5I+_Yv3zOj*!{>~3{ax_vNU3M7 zAKI$jO(w+~_!(z=TYSZ=#L5JODWwgS&C=w{gFm>Ih8XAi1bno>J2~R%QR~~3F}_>X zPxhn5_XOgXVLeXfAGFT{B%vIFUB#e@98C%Br-ZYv!L?v-y=P7b;ND?4*gO?^Ul4h~ zMn=q2{1{W|K9YE;KB(gj8l#_RI5DiLIQ;pAQa&$YlQ<(gRYXtPr1YM-X#K-Hq9CmR z&)=+q;oxR<#qQrr1HYxSyi=>vpp2Zm9gQ~&QCWD?>GPIq4lqoj3ec~ftGm+)#&+65eARz4E;V}qbMc8#EJEmBSDH8Ao z=`8P}zZpBdQ7S7X-6Jj?a(Pb;uwf^pOcU_>27G!<0a6rCz<@HRTWQx^8t0d}hhcY# z+b2{e)e%@;Tn@Ir7tOZ)fD{`mYgNXeR3nDARpnQ@Ff~2#3_pLki>)52QCRHQEO?(tZd1yaEGL01P|P9#b9 z|3b&itNu*!6Qq>&f2REC^1mf&Mx(x08MAJ}BKm2WHRa7;x4uDN(W3W~no z$p})~wlDKpg&0F~rQ{!A&Med0RNT@a3<}(sC$TZcR_fk+f^{%MM60n$Y^0?%GH=e4 zWDC4L+g}^L>L1lC^}yi%$Mr-2Wv#a$H*g7pO&N0c2-$Sl7ApODHu>T<&v8nX)!v2r zcMNdi0e)<*+~WL3s0S0|8Y(GytmFI6?40JT$ zG9tKZ`vA(JB%ItlJd?3b&7pgF)^An$)3TBipQv{ys*APip*{67{LDK2o@3puTXIUU zpM_O=zPKPh=6&JZSd7yWOw1sWwwKrY2w&#~_0{!li=-Lx@K8TWy9y4^a^?Xr846_5 zevFErwEnntd)m%h6{KZ^-UC(1_@i1D^+Z9i;=))7(sTbYwh0_Lpnn-8{d+$NJ$B*i zaVy~<65ce@FZ!x>_TH3MXSOV-Jjuejb-VLiwrG1%p}&V;{48~ z0i01Y60*+xc^&|O$}yU!6d(gj=zaH!kTIao0~n}XU~dz}O|eOQ74=n|bG9exTy2!iw)1`VE zj`nX+(MmdKdE1`SXBjr)wII6~upG^f3f+z=3G1t%aFzQ*3QD`Ux9gY#3|=0v){yc| z-4ZlQjTd_!QxJh^lt1o$lL>3O_|##y9D&MkNc?y)vaDI!uVh+|n}ZP9+?Ry~s}h*) zJk6_7mwT7Bm#6-G@1|C)=_H(DUAo2{nEL?y$@G(W)=9LlAs=aMC-=rB-zbHl+ z=g!s#w)F{9WsZz`FkWL)gD`*BstnQ9SCh3$BEpwL_qW8&7%NI+5v7aUT2FZ2rN&Z! z-@80~_^snrO!ZYS)x1UP8%zuJd4>A?`r^YmwH#(D4?m6`brMLpc`u(>|7Q`=)Kk;* zSqXcJ+h3!3Ek_bo7I_L632=_`t}0T?jb7BzgdarsSZu!#mRUUaJoRS3J*D=qKyNx>2H`^pC1KHqEX{KN1BP?DhRB_^ ztJ?z8Rz&JkW|vySr>2zA`-NS9hWDC#9Dz}jMoc(jnZzHrC5L%Vx)Rc$JjHBx#Ovqf zsa#%p)q#hzTnuILaEE;-XhYq<%eCh;rz1-Pb^`@M`11z$q0zf_*;u?$Kv zJXmV5HxU-X^+hJJt$aNRBPWr$8nM2E&JH(BV)~jx96gEl3F~{O8^^ydD@U+~6WnJ| zFAIrJs#k8=rh8pkR~$`IalS-n2HGq3z>u?B*7?2{-q2P)1TR$WW8FDj+lQO$>BEeI z8@hQ6Y$>iVZiGetGLsHy^*KmwXT891$9F`NgID6z)45VA=YhG|U+2aBN($K1)4jD{ zmsbm{G#b=o>MHC>XW55M|48}tx?1HDp~b{!`!|exE)^{2HUxwvzUB z078o7v8#yO#=dg(G*$$snDpcN$GSPXMIXT=8kgP}VX?8QW_vBJONKkD+)&+^Y?arx zp>I)k=Q)yF8D&r@;VAT3$)5lo$h1EbsD?X0{L{qdSL&hI9Lt@b)`3mN==OLd@ zAs%eg>T=nCw_n$YMkgI-iCfv{BtG}Pi@fYRRTWX6yRY}*B8kLD58>NTly;3>2=Q!+r!p^B_)02Mn~hn#euzF#uh<~A)VYB#Ra{w6flKl7mkAkQD% z%VmEA!3;5aJ-#fZHt_2j-rCXQ7kSz+ncUiaaX;_^GU8))vSz;8*HhgycuEI-96|eo zm0GnwjmV%OvSif}D+%H7R&}nEqhoJu+sSqu@*7o2?ON@-XN+F$uxWgqKJY7sg8bf4 zG>(StXySX0FV$qd(|ZMu%;#it+4a`vdz;6SL#Ji#S7&QoF$0&mj;1+J4a}&^q1KyZ zdEWwt)&jPZ@>UxL=wHriLWV2O8iqneT1EQ&$Djx-^oA^I!7*@NYGTH_N;wC`w?q3G zk-@R=e#RKx+UVIxZzz3H_EE?C<)XLe;)u>rer%E5<0JnryqyE0Jsnfd{N`?Jd-#u3T57M5vT4XF%>|+fHGZL%UzS6e?lD8Goz5-&E0u3!C z7`WrYIp=sj=@t3Y`8Z@oPxE=8`dA-lhe2n#Yb@$B-;O}qePj4Ut&o&I4#a)ei^-zDU8|}in4OB<|7_5m-}lE5lVapC z9CwPxU%T;QLN`=D?FLZd{V?_3A8j*Mej55s_rg;E>=f zCFxMN26C-oTE!qSkGkK3s~)81sP;YEaUy0v(S-}aS{v}flh-Yq6vxV}C6-=f2D#&< zSmG?+wY6F2fDLeIP)Y4&U$U9Vizda+&c=J&#JpXxPzdKg%ih}&bt^($-S=NtGCp7> z*^1k6LYbW3)Ic4^^Gly@ul2tOnQ9mBAqo*S6!()Kat%&%|6>r3E+xQrA~5Cg!;C6o zT9xqR{Rz=k$CQd)%LH@m&9cM3JD_k?%golP0^zELeC2;5BM-Bpn%dIkY&9ePQ~8gY zIGC8Q74#TsxK{a*kjEiAAU>oamx5D?SP;IFD#Fc~6A@d}IzvsidBS*I1&K9bAl0hG zgC{88*hjIx8aEmA;Wn`x`I<=S!{uj1b>w{tLZlK_hJ>_QY%Vij%vTsJ2bQ?U-+Q~B zci7D?{uT@7?`eZZ@>7#9yxkl0MOPyb^9?x*Es6*)hrOKDD@$I5^w0^~_gDaB;cANw zKfD-K#J~+{=_hG)@EkJ%OpDRI z{MIeFv?NuD&3tHlRj6ja1!dIsV)N@`%pfP6IJA#mXJhll7F{L%evTQQoNoZVZnE#~ zZ+ms4dpC=hW8k?}3MO(EM-{CvjUjSRfvQ52KbxQ`4YFFH&o3FPqgEr~XOpspO9+n&NDJr#W|SA%?j~rHUYRx)L1l8#nZ=G*zwmtUo5(5 z`GLEX*t8}r=EV$Gd;cr5)Sd4^Wn|U@$9u8f9=+K{SXEZ}y)A`rM^>n^Ob}vZzbs`+jwW!!anv0$9E~HoukcLl zPf6Ij_d^x*RB9R3tY+~ zTB<-rWIw4-{Pghtk-tAIxnwphKm&5aiB`-tW9E&6yThP~&mwE6{cJAFoTlANw<30^ zyAHM~oNNfCE_y=Yd(Gl2Z8rhN;B9Fd3(kkpKaWDRm;QR1C z=$>D~2+g%FmUeVhSIbL9N?Sa@agQs7JvqyG+No$!sWRe?Gu0dK0C`^sZFEA4nDb}{ z-^p|?i;%BR_oJ&wzAS#;KvHb0HW`rqXiI4M;zc@`!s^ zek)&XdvK!yzm(b0A-ZZ z*ufl~KJ(c@Gh*H(Ieo{>+TAO`ih6i4u*CLh#m!w5BrS?hbIIG;Hqk!WB297k8*e!G z*!$we_96J9uY<3I;ds@(E!=-$mj}9+AJXB?PN-`C?uSET%WJhY{o|<*RKh}m-Pt!f za#?&8LmFER)Hxj}th+%!AmJIIhOl{6CoQfdJ|rF52>srGi44qzF@C~QeK|83_MH6g zf8V5Eg}cW~blXZ|X<-V%G!Zcg}LBn`&R&o|lXWjs(Sms`yjL$n2 z3=0ORqr_F(s!@^lH%c9Z5i*lt+nxw%{M(-NI|7C`m<;T#4EcHW2{Xwy!Zf)67gg#B z#VN{ps~0N$@3?|U{yR-=UGIYU3ZI#~JT@m$KF;+o#oYR%Kry4`(xoA+T*SQUQwG)@ ziI7KMB}JPcL6VXoU{Lv&U*h3ihX?rhoG@5N&yqA9zmw^|!?hMF`z%@jqi@4pjidQD zjukW;vd)(MCs_PA01O@zG*G6-A$h2V(S-sEY^f)#5(BR4?%1^?nFmaJ`ve#hJ`$ss zTXh-IL&>vGym@rvQ#yb!2@EatyK0E(!o{k)Jb!@cvH@`!=33J8`78v((Epz3pRK>MFptvzk{-9egy#SLFNOs> z?Rt;l``#^i|NBZ?{=VKvj|ahm-)tz%N$!TnAIZCBU0wzYaQ3S{ePQ+;)p`64yb0|O zfFE1y!TA!4asN4a$cg%?Hpc8u!Uw_czwotzz4cQmVCCb58JcoMv{1D#WcYy-Rp%qr zrNRI9v6c!~DMO%%m2A%!!P{S8ytBj1V=Ym@_%_|;B9D*uLKlp0Tgdw+W?aOp;KN={ z%anvt^4;hHFw%TCM`i{FlSVlyuS0Ml7n!Au_?T%CDP}%(($&;y#kP22b@`SnR}=P( zZ>%s%QzbmE@~eB&x9$+l=TFPBE~oGPo6i6M*^Ib_RxWa3A+o?W!B!I7 zYgujf-!a4^hk+NF(zjUq)^bL;p6TkTTLK0dl|fqJ?bx7|0JABpaK1!!ZvR1fKZlM{ss=(t$7s z)i}zU)Fq0DAKBi=t~ zWk~WrSa^)0OSUl>@#!9*`z1xamCpPFoD7NT%VP6AUiK>26;0FT8rGbF%C(>?Lno zZ#NS30h0UWtwgHX$tuk)UvvL}SC?W0+ta`CUwl-v-fgVH9-a|vzMFqHGT|Oq`UF!> zXD@Q|{X@B(Y|EGQ^W zhdGtTU}F8pW}LL2)e+u@`wB|e=G$sZ0-PVV1)8K5+TVO{DRLxgtIQ5yRQ^3@6VR@l zstrt68^{O~e55|ep_YrT4QPe@`J`fxX*^(Nm;TFo1(|l=`OHVnQpF))_;V9uw9G1f zyXjQlZ4B_!X~c{oMzjmaV%}uvZXfy9)|UMqlaCZrjN>SU47~y%xFNF5>w4@4^OIQS zuZ}WeLPn`PcOlb7{^n3YrC5A)7ln3o}F0=vLGVT+Q z;x=t6J!%=HLo9uIHL8z%0^zX0?Yb_-!ej}+pMkEtx`bXmo2@eVD`8BIl_(XnVS)U=GwI->>or=ma77#I?BnM2P(@{dW^R^iv& z%waVj;ge|ZCBq%}Sn}3nx{Cq~Yhvx*t=SS1TJGQU3r9~4J2pyW-t?KpFH7KSv6ud4 zic=zV$+(O7G$ID2Pmuhb5F*W8?#syo9QK6jZ6d*V!oO(KZT zA5ij0$Vm$pky9r2VgAo%seZhQ73)T_S4veC8A=&o_k5Ws{GsX5xzP2(k!9-=C50jP zlgVyc%sf6ZtNTxw1wULwBFHtEG_DX5=NfGDh!^8+0B^YLe&=ytNXHD#h@b)If{#GS>pdEcSbYV^(dzV2aIv#Sd+=zKdhL zZ(z3HI~8~Zge>--|6x!6G1LFn}wi;Q;_M)*sGdrQewvE`|7Xc zlLug1Fa3j+?AG?0?FZc}cKDf_kmO9}cg!!~XNat3{~rSiOu!n2;dBD8WMOannmXom z3w%Ft56*KTxR;5X#|pJD9%$`4hJlMDZhObpJ$2_{*Ry|?mK*yNczx#Fl1-M0j`c7C zi5eGB7h?=uONu>6X8PS;uFU5KH1eNkz6QCJMnut;3z!SU0nZ&_#+X$iUl-5D{cU7j zzQ3N4C!(~NTj!%8q-MZLmh~7s;-e}pCE~RiyLaZyHdbf z^)mzqw8=vImxKB)MSm1}82sJ{H}s#LMmN5Y&9dVhY&5RfO{<|F|IE84L@^y0PI~oi zo3@XX(x2V5x_rHnzj-k=@rD-S=HE4>%ZfDx43)+Y@Nm$^7K#WPJQX=2ixN^ie}gIT zy_l1o3SuW2LYTWK$I9447hpH-VEO#}kz%Ua$}(K@h^V4`diN3L#eU#=9uYdejyKJER>rHeBQ zBGv=^c^WQDhJ)d#OtQJool?@`3>B>ijRbk0jgIbkbw#iF@-Jzzf305h-3&8yOsVp+ z8u?lxVzuhoY<%e_=sL(k#yOW5MGJ0;2-|+lP)vNi_!B23%|;+7jz&?IQ(^sg%YmTq z!P573w4J@Zo`7;H1R&EFE=wbS9!DI!EXaw;Cjy+yI|<(1gE8#O_lq)pn(mUxE{!uM zNO0S8*su*Y{F5iApa1HJFfeuA>O<=i%cN9A3F&jS|0x14^Y7Zcac1ckl&}CY>>(X; zpqweqLI?f$RW{YTjHT?Ug5jCC(X+lFTs%2`Tju2IurWc-Wun{{9frzOgRm~mHaNz2 zQ*blUS9mXTG5U{}n}kDLvfv80tbH|+$p{)ACp$J0)yTZ3QfBf@IRBF3N!K54vd;~( z0%>~$!@k%{^lL~A>`5D1#??fK>4=O4J4Hs-pG1iy?Y$8eYjQ+jIVIqTQh31a zRhUCg0GYT}Z2BHBS(#u)r1_48#+F8+k_GfCDCP#FLXpSo=H{lWPsvCN*xYA)!6>%+ z5WeJpOgt}9uGFbBxp;&MR^->8;vG0D+46pS)?C87uP7JfZ0Bx)HQ0GPCiX%vb&^}2 zGUq&`gT)=aUOMnt#0$pCnabT_(5kHirXM=TnD%bUrkgh42_}Zho|+%#M+{8Q&JQ<4 z(MEwODx6I`c4GH6Ro>0#y=HgtVFl?0VX-1UDP6n!6tT`mKkwPE#9l06OX}sOAt719-?l)bo)7LwkANSg= z)(;P6d3l!mK=g)IPNUs|Xz9(dnfP@kf$zGBr(%06Wy_MJ&{8WT@kM}7t5PNOs*9OsN`K zT^!2x9-OW08Va09T}^f2B)i!0TwMQ`;G7oR5gF@7qN$+wUvup6?$9*r^Cs)Futl`G zwCAJkeJR&G%k}tfQrpYSw#{16zzm7JlXYi(A`q(MP*gO)?DU*}3;q|e|CT@D;AN}x z-d7%!C)b=ualQev?0l~MwkxBzK3b#QNfdtX$kaDx)c%aufl&PNKImHf1=lnVxWWxUz18KJ_r{>TouQfIYhR z(hsVXd~}MRYh$1Bf|tLJf&QSI&p{^loctdX#G^Ys-czZ)5UxYUrF zFCC^jU9d)NX>FMXBNrX3%WVlCE^Izuo7--2wTXRg6FrYIi#Z0rSA=iK-duhn`pMbEN&||LrO&j)kyKBKT18VA# zB!@E^b=;FX&enfpf#%nXPJBPikvOefwnA|O8*r`=w=W&XatY-if#qwZavRzUBx2`B z(M~8DsTH<-y_Wysg>7CGm+MdCe9^=gyS3lJ4%%5CjS(rI3OcBzjA3T3>L$njCv_{J z1%`~>xUYJmaQ~_+Jel)ZyTVws6#q(uFu;V&mQ4ccJM{_(p->7|^a$Kc5&L`C*1!E} z>2~n`sB186D789rJ3=~lYI|r?=B2#!>zLy)Wn13oWW$znOn2hPl|_z;UN-iT<`-$h zUvb<#fstt=Z)rJBW?X}h9*VqX4PHUft6{pHxTaw{c_&XT+x$rX-J0Vg?20z&F~yUg zoEYz^=_@0&W^R**S96X`=T7U`NM*)vW0U9S?N7UIcz3jm`^UwoH3+5QG#cy^8zOcU#I zcF!tj;9Juln<0HaHplcJnDkOz6OiwaC`{<+MK>z)mc5Lb3A^{Q5mFgZ_xMhC1<+0h zal(a3H^7FYQwSq*_$K?~#iuFDQXWkL}cOQ5EfO*}OxBn447l?jf zRA)XFU)1sq6ikZv@5;b)467=X$LKe4_cP9qX9{<~1vAgQLwNv< z^l&!U_ViJJuV1Lr|TNbB4-siX3@@$>S}^5uUrRD!!5 zV+8_(wx!&rcFQ2_k`0o6MYlecL=vPA+;6sItV8KVi*P!h=9V&y_ME>vr_;MhwXmBQ zePwM^bH7uW1l&ElPBA&;brUz0FCfkj6)-R}>!@!W!YF}H*z;X3J{DYEt1=}4=HksK zOCFx-7~bB42i@oKyAL`SZ;2&WQ3jvUHS4s$=DdS8A1#kGpWN_#xaHD)8uS__j}#g+ zJG*_<<9K+X>TFg=u$m&TaqGXjL<+P$g2WIzn~5XZHO^9@g>&myeFs^lqHy|ll>QV8 z3clY?p7CTqU0}LhxidSYoxUgQZ*HKlz0;<6z9@uw(%D9H@sQUciQKjgW7fwd9H>qD&YqXlDa^4*yd5cK;$}ykeCtg=Nl}O4z#NP=_#MIeKM|Ow6?AGywL{{++{qf_CRjODND|li1L^pxD?%jQbkm8f zN$b|&kBsdhz-rjn@6xEDfBUFDMEcU_Y&*Xm zPqX;FH{Tr0lIuLS$6t8J3utgysdtuq-g;DBt5qpmsaR>TptQ8=7F{!3X~%&M=C+P$ zWjE{pW^yj+9KPlaRcYu&t=u5DBZ)%B6m?^m&4-)IGu(TE`{l|XYwG8?Z$8(PqVUUS9Xdw+VYlny zOZCIM#CWZ|@uv4+ms2YC1w>^>49%}Lx41#?r7LwtpM4%HZ9Hf@s6|#sq|}{6Z|~>Y z54-9xJ?DNwf{|7g@Lv`A8e^$@zG*BxY^~gCn&|3*XjCKXig^)s=cDT{;V1Tn2=DFF z-C!qs$2=ivJj3SZx0il!#*HE!3sa-7!Ft{ifdkf=F2?T|I5yrHkm|b$I9B>v_>q;VR~5b+ci=XU*JtoHzGreH&pPzd(6? z$vCBYk?^DV>eNmCYi30yo!|_5Fmisl0B0ZsEs%|>*c_YXSQ?VvX;o~%>`$qq_rRUn zn^wCp`LG^j&&Ii>)yOAyt7U|8tHB(|h2l-*A4G!7yCPJqAcjN|I-|TRWeM>ik7#;0 zn3cop{eXc;fe$V3RYf%C?(4`6Cewa$FVGup@T>pj4f3azTu@lBH}LYF0&jlh#_F$A zplpK(v>vAKTc-Lfif=~7r>J|!)=f0$mrupZH1fGQK<1;~>>$R3qxQCqAMu0rC&lGG z^~w^wFdDCCY||=vb#sh!48*@rRkuwY8a072yr42#x)#)nX`eR#w7}(9GxwHB;@znI zHfiye)W!XF{YyWglhoAbC_HsjQ`^UW6Kps!Cw`#f{`Xv0xB6wu=VL+~6%L1T=MQhW zN&F6CvGnG&R>H*Fp|fV-rRxping#Vw`PUn-zu)O6rjm{~5A&sl4;`7n!TO&%$`zID z7KSsS)~O#QbF!XX4M1`3do0dU1#au!AEqGe-RXY*K_>UdES|5>HT0+zobk52W>-ZZ2@ub z`R<6UT5prg{AjB9h7UNQf{`|>m# zh!mL*!SUr}7dNSLmx5=`&e3yrltjiz6%ZH@rQv4*_dw^t*Zk1nIecnbq)`Q4yBO2| zspdMPn(DeVB7#U2q)E5X2@s?RN)-eW0Y!=dl-@f?uhLYIA|>=9y@p;w6Y>Jmd+#q& z0zxQaC<1d6XJ&mf>zi-Z{G9x_e{$Eo=j?OtK6^jUv-j$?wfVMli08v=^|FF!%|qbuEL1%-!nx54U5+th}M zH69#pmUKtAq}Mepez|qubnl}$7D8hN&G=Wza!d)2+&$VZ&mkWcxT3)&g=fe}-dmrt zv)xmn%YR3a{3LGL^2Vm^nu%ManamR$16%wTOO-`-Q)Rw?skHxd87db_Cd5?bNz5+u?J`kX_OsK^oQU5+`Dev{i!cZ8l4rTtJq$O< zp(t{BoxCs(Vu;-o`wJ#4W!u{Nrv92{M;;Sd;g86i*WkqiAUNb99qg^9)Q3#jbWSGQ z#qCE~ZlBpJjMDI0Bo6X&z0CXm`PMpH?ziI>$~T`o(v!zfi1D9$%ycKa{TUf++-VC> zCmX9{FC6+5&@NkTTkl9vZI?#z(yN;(5)r19P4Y0RbP2wB>?VhVP{VTQw@75HNaPh# z_bR#OIl{*EfoHLCnX`8=eM9`NX7nRhpKffL_U{9Pv_jYEjA`G|wu{T!s_2*F$u!F4 zkAsDdH>PVz7W~dm*ZtiUjv@@}g%|y1ZL0<2T>9JCHZ%DlZ>O=4h~?UiGx6@pbfPI% z$IyA%kQPHZedJ^uwrDeH(7v(FX*eMGCw!WUW7U05&>!1X_Lf?AO6 zY}EZlA>bq{>i{3S`m`YY(#5LzEX5yj`Ptt#-S4T;!RsiZufqqrcl)%yot=`%*3)0AhZ!@SiGC}AaQX3~V;_D@_$HYNr27{}AzSTbp zZUJ!@ricgBtzF(f{CQdMjob2yDL#Y4Q`H0n-=0JQn>{Q28<-aDRwhAzD(+BnkU+t2 zCDp|DCn5WLPu(Vta!RUlxGmtzi?fqKLOJgoe1+n_$_4*Ln)rYGE5^49P=OP~^jEpK z3hkfxMsF49zvUgvGf)FGFdt{H;FCuc*X*z`EPy#=jA>vd*TP3+O%4wDpmOmQjKg^f zB%L>G$O&e{c5{I25{<6F2KsvnS-587g2IA6E<+u3mL&LUagJ=}``lO!I=pFyRav?l zYm?!8ZprzQZ}Oz@ocbnFb{H|5OGQJP9fgw1fxtNfMGBj6Q@ssS!fw>ld>eMc~*F_k`az z$9P7)7C&F7#3UwVdx=q(>SrBUA*k)Q!tQA80$y+N;sr!7Fh2q<$$A|#9Ia7cRg!J6ZQJDV%6&NGFTc}l%>cu( z4xij@lhs9^$(CI9HW90l@zkn=rVE9W@o+vc?~uRDLuXapk%v-$e{4 zt@tsqNEzdIocJB2GP>OGgrwu>6m3FhEtXbRU#1ai3xB0-+UhV)B~9yDV%nh=3}D`( z`yV&F{SWy)&JJ9f4b@6c+@AC_w5zI{Iw>w>Vn3-GDUW0oEm3QSyUF7jqTTii1vv9d zGnW)?DL@KqEMH}i31EzQmrSi~j%o0DX9t}113Gsu_qASWX-rOz&D~>(+K5|mFImXy z7@W#&vhTF#ceDj44Zq2)7m6HDklA?v=wqfPX@Ec($-T=4HT^ua-iJ{f{Ze4&lIEoD zZB#B)(H$8BTC7~bs`&{#?3!~%!qxgFotYd9*%7P_Mx#k-Ls;*ZC%$n%ry|^V`zoT} z5$$>PK4%Ci-o9`K_M5H#%p2J2yb`OHFC{#qw{Qd}N8CH(1Z%N)RU7(LB@CIVJO6|B z%DX6|H^~o_l`QD}AS8_g1wI0z>w8|b0*J|ORtpQkPxssNVO`98H;tp zM0suuOcI6T)m6UX0wS?;9Pv`p<&4t!DrLFZP(^FbcqTP-rU*d`h1-Zb)ip{_i8{!R zPzgwu@u1xk$v$YBy0G22d5Qn&5>iEX)Ha1Q%)Qz8Ex^C~>%EH|V~jt%R95Pm%-cQ< z8|6DKr6oU#cc`gaG~%c2QQ0o8Y3K#h8vC34S@Izxp0Y6TpeBscvJ+VIf;P69D+P(v zT*glNKJ0qPs1%*<0Sl89023^7?q*)BJD5D}4W3Hc4;yt!R9~fd+KBxwx#ZDJF{tOP z1?*i`S|Zz95GY5ma6WESN-ry>gks${`Laz z%Xi$1qW}tad6*he?c>q*=ES#?53FTf90hPu6&_pfJ**q*E`oZc{aXZru;ct)6nMrw zrm|gAVcd?DfM46}ssF8xJm&n-y_L$X~Zs?Pr5PZ%-G@B!)s|&VHLK?}B{y$g@#rrts07%@Rluj}Yo7V9F&2n;LX{FLcvOQN zbKHCk`4^0~`Kh9c8qQ@eDuI-}Pfex7UNDH7=~6m|4?r84*se!F8SSipXSv;Nsi}3y zwV+{QO63-P*E-E)s6=ayjCT;Xy%RMa4OB^ssq1P<2>1bFAKC40eG*Vojrf`CW6Uq} z6Ny5V5@U{@*3g{!;s3^J1FBCtpp^@NekQAT-75YD&FzH>+%*sJ5&$>R6@LTMNz_nZ zUwb@CgH3tGb%(dduHY25c6#q?)WlFK8Dg}o>N!9!MI-)0j)CRRoO|rrDs}#rZVqy( z!uZm(gZp-3PvjTU=5Xg=JRqCZ8OQ^9882~S$_u_*#7uv5_>LY%R$mKVwFF;NY#g+g zqTYorr{qDb&vze|=wg#SJ6%td-ejy`Quqzkri#9qJ+L$d`Vv^Ce}6zM!Q!*uvO*fk z98daI?=_(nF+O~NQGTSWD$fSx3BF~2mLTVuGW7#A!2!{0SfyAhVFB4*6amgvHQloZ z&@Tb}p|-{?`lOg`pNdzL?G0N%R2Uz(i_OwSsbVW3_946 zTu{sF6g&n_xq1jA3*a)eANu($Abz*@oNm9EaE9oSPnbtVGLvg(HhAD12JQ7q3ck@^ zpidr96lX-o2ktYespVjFKfWZJMU|*BCIYlzImj{3xdmcd82PXnhJ+U%FPH$T<^3>I zVxf~WrV9sviw`Y5JkGLd+YdX56=nof9XMZ2>PA;<49o#%+F>SK>n0nZ|DA=akPR`q zqZDJw9UB{1Bbttx5?!nqoyUrudNrTbLG&KHT-@Kz!x8#*8xn0U^+G~Vk@3BHicGGa zT&om!w<(Afg{J%}Jo*8J&(cEgg}m!yPMKP{IsC_)WnSeL?3UvzvvAcIDp@LGXqHDV zz9PxyF`p4BQ3`Bj|E(wq^3c4qq7c-6fwK+zW2-!Hq5<8GEK3`T05fkOM3?Roays3{ zr7{Fo4_WN{_Rc(}44_83)W3J*?lJ2fUH?ChJfBcEjyu)LKo zMDDWB2|L8x=fVY79Hf;z52OgIpXL)|JW(sb@QY1{0Js4_OMU$4#{Y@ck>?*}LA10i z5_&_Hgt*Y;GlS>_XCBakUMcnWad$HV$~Sp{ zd=uC{*@5h0*UWGp^!6;^=30KLbKhEo)!Ps45do^8e&=9clE7c6Bml69L9ZxxwHcbW z{-zcC%*aY)p7G> zWm3}EKc3QW=2Z&~qXf?u-6}C20+Az$IU=s_M4!H|3rE>oJVV8j5;f!9-t2e)e zZ5bOiWetje7AT}4mLHPy#zBY9&&Zl+uKyWu)8|?e53sv1?>pm9Ajf13>AzDjKqVsX z@W?HJ6?32~=Cv;fq9TJ=agAzA z01y0`cO&pSK5F~tJW>o34tWONt_Iv}2Xwu`(Q$GGxJI70Z8*`b+$2TUI8KKBfGG2y zQAaUbz`I8bZ-~XdT#9QQC;T&bf#7HI2aEte`G>NSqcO$_i31rosMxCXWFsRFM^}CL zJjw07NWO-^j+Nc?5pKCUw;w2^45DYy{6ahva_ zKb?rs(4_f05aj!8|Kpv{XQM+AgSq~H@~Ib5M4aapstlx#aJBa~7O6ya+9H^Ai>AyJ zr=yyW*0JMr;Yop9-DI2R-CnOd1a}i4YtIl%}oGoT_)q=T|TVt;WC_Vw!t;o!|e`*yvoZ<)+o>V>K zx>q*EY5jp)^21CB=5i`<=>wwBPbF& ze45P|sT@DG{-|3PO-JL{b1N)O=4=@a1i%Q|x z$vUELChsBErpx#DZhZMebh6q4J)Hd3oPpesGNau%-xKMLyfR$FQvQOw`{rU6cIHmI zrtG3|7o8d$%G6Xp4SjhxTjSmG5%SSJ+M3NBai~mL=8yzJL!|oC?6QuTg-8Noqt+>X zDs`1^6N`ZN*g==$NiErcr_uB}Pu7bNV8ECeyG0?a@C5Dqxr4_{lj^htSMVdRDMolb ziqn)7Q&-{JBNC8~(qyYoqGl9sw`(PJsWiR84vq%%7p33-X^o-edQdL0QHeo-SE`1q zw{7T>$Gg;tXoashq&!?F()w{`nc2lC*SI7--m@lDYF$vNydrD}EH#cB=0FeVaC6)3 z7Yonji6J`Fzndt`=k3c{nA{ZOGK24WlL|EG_4>guf|8jRi_i=uPNwlQK@Hd=x_BADkJ{nsT2 z#NA1*HAmBH_6_v}nKwB6PH^Y}Fp@unGng7L{X$Bq(A{I@O2&))3DQ@|;a}vseKoF! z%F*EC&VTNo@D4@g6@bw&a{cjeOPXZ(dHyE{4uBiWmzJ1azf`FlNrI(p;rP(d(7eGw zJ}=&c&5jp$YZdhZ+p>0sIkk*`gfK~6)e0vcZlQsC^1O%(y>@StM!Po*NK^K6%xn~h5xd~*Kpj%*e_5iM+P$qn3p(7tSN5~~7UM}oKMSge)`6%T4L;><~p z&Pe=WGaG?b@dP8&b=i+{Ra?SmkI90)sG<^loS!P;?2+{GdeG3{F477;?Y*-dEp_y( zI*r^o?;ql5Q^FeMI`Tjvv=8k_O91IXpb* zuG`WPyB2RI*o?FO*xBnFOmysxl!rj>XzC`tEnoI)3d-P?1~ZQe(XjeHYy&-}@z#_p|KYh0 zp8&8R4@PlIMDuPEc6yHCzhY$NZDo%#DhO#`BwCbNI&|>unxLU9lPKjJ12l|viMAV2 zPbH-m>(a$m&jy6|oyBI0-nL!ds3i+-bP$$^ZIsQHJ4R}hjCiA3d#9p-X?~@5X)zAT z$t)fLr5^4}PO+AYH`?kTa~5dc@ASfk}MXE(1S9I+h?0j0VDrRT7+ z%ik5NI^)4;0IqAg9^NVoGhtkP64JinE;K(sU6}FM+ra*jf3Xb^9mt*a3xZfaKz7Gp zMI43gV+I^vlruNm>pri$5juCRV}K!bqRUa^4)sMo40m_QT1_9;#O;pLz>jr-wf7&$ zgPBa8-CPUsVl&?J3TY<~!<|rN^qj&&minM>NfyC$6npPsHO=+tc)6r2?nWs*O||>q z(S1c_{5Y%oX75%%!}`hpdSc z(z{%Xm7C^}0-nE071pTin3s}`PBxb+S-pSU1ew@`w(GL)L!?hBFK4P0ZWEkoNNltA#AN>p$Lq0@$E@o~uTYApglfL+e))!e*KnvOR)f0s1znj(OZL^ zNjh?UgUe}hda}4N49^=Sr8f;BgZNx6d2yMF^oSuThG literal 0 HcmV?d00001 diff --git a/docs/index.md b/docs/index.md index 960b968454d0e..c0d97f378555e 100644 --- a/docs/index.md +++ b/docs/index.md @@ -18,7 +18,7 @@ Users can also download a "Hadoop free" binary and run Spark with any Hadoop ver Scala and Java users can include Spark in their projects using its Maven coordinates and in the future Python users can also install Spark from PyPI. -If you'd like to build Spark from +If you'd like to build Spark from source, visit [Building Spark](building-spark.html). @@ -32,8 +32,8 @@ uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scal Note that support for Java 7 was removed as of Spark 2.2.0. -Note that support for Python 2.6 is deprecated as of Spark 2.0.0, and support for -Scala 2.10 and versions of Hadoop before 2.6 are deprecated as of Spark 2.1.0, and may be +Note that support for Python 2.6 is deprecated as of Spark 2.0.0, and support for +Scala 2.10 and versions of Hadoop before 2.6 are deprecated as of Spark 2.1.0, and may be removed in Spark 2.2.0. # Running the Examples and Shell @@ -84,6 +84,7 @@ options for deployment: * [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy Spark on a private cluster * [Apache Mesos](running-on-mesos.html) * [Hadoop YARN](running-on-yarn.html) +* [Nomad](running-on-nomad.html) # Where to Go from Here @@ -116,6 +117,7 @@ options for deployment: [Apache Mesos](http://mesos.apache.org) * [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN) * [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark): deploy Spark on top of Kubernetes + * [Nomad](running-on-nomad.html): run Spark on top of Nomad **Other Documents:** diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 807944f20a78a..dcbdef529801f 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -24,8 +24,8 @@ different options to manage allocation, depending on the cluster manager. The simplest option, available on all cluster managers, is _static partitioning_ of resources. With this approach, each application is given a maximum amount of resources it can use, and holds onto them -for its whole duration. This is the approach used in Spark's [standalone](spark-standalone.html) -and [YARN](running-on-yarn.html) modes, as well as the +for its whole duration. This is the approach used in Spark's [standalone](spark-standalone.html), +[YARN](running-on-yarn.html) and [Nomad](running-on-nomad.html) modes, as well as the [coarse-grained Mesos mode](running-on-mesos.html#mesos-run-modes). Resource allocation can be configured as follows, based on the cluster type: @@ -38,11 +38,11 @@ Resource allocation can be configured as follows, based on the cluster type: * **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`, and optionally set `spark.cores.max` to limit each application's resource share as in the standalone mode. You should also set `spark.executor.memory` to control the executor memory. -* **YARN:** The `--num-executors` option to the Spark YARN client controls how many executors it will allocate +* **YARN or Nomad:** The `--num-executors` option to `spark-submit` controls how many executors it will allocate on the cluster (`spark.executor.instances` as configuration property), while `--executor-memory` (`spark.executor.memory` configuration property) and `--executor-cores` (`spark.executor.cores` configuration property) control the resources per executor. For more information, see the - [YARN Spark Properties](running-on-yarn.html). + [YARN Spark Properties](running-on-yarn.html) or [Running on Nomad](running-on-nomad.html). A second option available on Mesos is _dynamic sharing_ of CPU cores. In this mode, each Spark application still has a fixed and independent memory allocation (set by `spark.executor.memory`), but when the @@ -64,8 +64,9 @@ are no longer used and request them again later when there is demand. This featu useful if multiple applications share resources in your Spark cluster. This feature is disabled by default and available on all coarse-grained cluster managers, i.e. -[standalone mode](spark-standalone.html), [YARN mode](running-on-yarn.html), and -[Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes). +[standalone mode](spark-standalone.html), [YARN mode](running-on-yarn.html), +[Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes), and +[Nomad mode](running-on-nomade.html#dynamic-allocation-of-executors). ### Configuration and Setup @@ -85,6 +86,8 @@ through Marathon. In YARN mode, follow the instructions [here](running-on-yarn.html#configuring-the-external-shuffle-service). +In Nomad mode, start the application with `spark.shuffle.service.enabled` set to `true`. + All other relevant configurations are optional and under the `spark.dynamicAllocation.*` and `spark.shuffle.service.*` namespaces. For more detail, see the [configurations page](configuration.html#dynamic-allocation). diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index 483acd347a826..1eb40bf52bedc 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -167,7 +167,7 @@ sc = SparkContext(conf=conf) The `appName` parameter is a name for your application to show on the cluster UI. -`master` is a [Spark, Mesos or YARN cluster URL](submitting-applications.html#master-urls), +`master` is a [Spark, Mesos, YARN or Nomad cluster URL](submitting-applications.html#master-urls), or a special "local" string to run in local mode. In practice, when running on a cluster, you will not want to hardcode `master` in the program, but rather [launch the application with `spark-submit`](submitting-applications.html) and diff --git a/docs/running-on-nomad.md b/docs/running-on-nomad.md new file mode 100644 index 0000000000000..cb9e15cce7445 --- /dev/null +++ b/docs/running-on-nomad.md @@ -0,0 +1,722 @@ +--- +layout: global +title: Running Spark on Nomad +--- + + +Spark supports using a [Nomad](https://nomadproject.io/) cluster to run Spark applications. +When running on nomad, the Spark executors that run Spark tasks for your application, +and optionally the application driver itself, run as Nomad tasks in a Nomad job. + + +* This will become a table of contents (this text will be scraped). +{:toc} + + +# Launching Spark on Nomad + +To launch a Spark application on Nomad, set the Spark master to either +"nomad:" followed by the URL for the Nomad HTTP API (e.g. "nomad:http://nomad.example.com:4646"), +or simply "nomad" to use the URL in `NOMAD_ADDR` environment variable or "http://127.0.0.1:4646" when that variable is not set. + +When using `spark-submit`, the master is set with the `--master` option. +In general, your `spark-submit` invocation will have the form: + + $ bin/spark-submit --master nomad[:] [options] --class [args] + +When running your application directly (which limits you to [client mode](#client-mode)), +you can set the master with the `spark.master` configuration property, +or using the `SparkConf.setMaster` on the configuration you create your `SparkContext` from. + + +# Prerequisites + +When running on Nomad, Spark creates Nomad tasks that run scripts from the Spark distribution on client agents in the Nomad cluster. +These tasks need access to a Java runtime environment (JRE), a Spark distribution built with Nomad support, +and (in [cluster mode](#cluster-mode)) the Spark application itself. + + +## Installed JRE or Docker Image + +By default, Spark will constrain the tasks it creates to run on Nomad client agents which have the `java` driver enabled and at least Java 7. +The tasks use Nomad's `exec` driver to run Spark scripts on those nodes, and the scripts make use of the JRE that is installed on the node. + +Alternatively, you can set the `spark.nomad.dockerImage` configuration property (set by `spark-submit`'s `--docker-image` flag) +to the name or URL of a docker image to use to run Spark Nomad tasks. +The tasks use Nomad's `docker` driver to run Spark scripts in a container created from this image. +The image should contain a JRE, and optionally a Spark distribution (see below). +When using this option, you can use the `spark.nomad.docker.*` [configuration properties](#configuration) +to control authentication. + +Note that when using a Docker image, you may want to include the Spark distribution directly in the docker image, +you may want to include the Spark distribution (see the section below) and possibly even your application in the docker image +and use `local:` URLs when giving their locations to `spark-submit`. + +## Spark Distribution Location + +The Nomad tasks created by Spark need to have access to a Spark distribution, +and the `spark.nomad.sparkDistribution` configuration property (set by `spark-submit`'s `--distribution` flag) +must be set to a URL where the spark distribution can be found. + +When a `local:` URL is provided, it must point to a directory containing a spark distribution. +When `spark.nomad.dockerImage` is set (see [Installed JRE or Docker Image](#installed-jre-or-docker-image) above), +this designates a path inside the docker image. +Otherwise it is a path on the Nomad client node itself +(meaning that Spark must be installed at this location on all clients in the Nomad cluster that have at least Java 7 and meet any additional [constraints](#contraints) you specify). + +E.g.: + + $ ./bin/spark-submit \ + --master nomad \ + --docker-image your-spark-image \ + --distribution local:///opt/spark \ + --class com.example.Main \ + example.jar + +Alternatively, you can provide the [remote URL](#remote-urls) of a Spark distribution archive. +The archive will be downloaded and extracted into the task's allocation directory. + +E.g.: + + $ ./bin/spark-submit \ + --master nomad \ + --distribution http://example.com/spark.tgz \ + --class com.example.Main \ + example.jar + + +# Deployment Modes + +You can run a Spark application on Nomad in either "client" mode (the default) or "cluster" mode. + +## Client Mode + +In client mode (the default deployment mode), +the Spark application is either directly started by the user, or run directly by `spark-submit`, +so the application driver runs on a machine that is not necessarily in the Nomad cluster. +The driver's SparkContext creates a Nomad job to run Spark executors. +The executors connect to the driver and run Spark tasks on behalf of the application. +When the driver's SparkContext is stopped, the executors are shut down. + +![](img/nomad-client-mode.png) + +Note that the machine running the driver or spark-submit needs to be reachable from +the Nomad clients so that the executors can connect to it. + +In client mode, application resources need to start out present on the submitting machine, +so jars (both the primary jar and those added with the `--jars` option) can't be specified using `http:` or `https:` URLs. +You can either use files on the submitting machine (either as raw paths or `file:` URLs), +or use `local:` URLs to indicate that the files are independently available on both +the submitting machine and all of the Nomad clients where the executors might run. + +In this mode, the spark-submit invocation doesn't return until the application has finished running, +and killing the spark-submit process kills the application. + +For example, to submit an application in client mode: + + $ ./bin/spark-submit --class org.apache.spark.examples.SparkPi \ + --master nomad \ + --distribution http://example.com/spark.tgz + lib/spark-examples*.jar \ + 10 + +## Cluster Mode + +In cluster mode, the `spark-submit` process creates a Nomad job to run the Spark application driver. +The driver's SparkContext then adds Spark executors to the Nomad job. +The executors connect to the driver and run Spark tasks on behalf of the application. +When the driver's SparkContext is stopped, the executors are shut down. + +![](img/nomad-cluster-mode.png) + +In cluster mode, application resources need to be hosted somewhere accessible to the Nomad cluster, +so jars (both the primary jar and those added with the `--jars` option) can't be specified using raw paths or `file:` URLs. +You can either use `http:` or `https:` URLs, or use `local:` URLs to indicate that +the files are independently available all of the Nomad clients where the driver and executors might run. + +Note that in cluster mode, the nomad master URL needs to be routable from both the submitting machine and +the Nomad client node that runs the driver. If the Nomad cluster is integrated with Consul, +you may want to use a DNS name for the Nomad service served by Consul. + +For example, to submit an application in cluster mode: + + $ ./bin/spark-submit --class org.apache.spark.examples.SparkPi \ + --master nomad \ + --deploy-mode cluster \ + --distribution http://example.com/spark.tgz + http://example.com/spark-examples.jar \ + 10 + +### Monitoring Output + +By default, `spark-submit` in cluster mode will simply submit your application to the Nomad cluster. +You can use the `spark.nomad.cluster.monitorUntil` configuration property (set by `spark-submit`'s `--monitor-until` flag) +to have spark-submit monitor the job after submitting it, and even tail its log until the job completes. + +In any case, once the job has been submitted to Nomad, killing spark-submit won't stop the spark application, +as it runs independently in the Nomad cluster. + +### Spark UI + +In cluster mode, if `spark.ui.enabled` is `true` (as by default), the Spark UI will be +dynamically allocated a port. The UI will be exposed by Nomad as a service, +and the UI's URL will appear in the Spark driver's log. + +The Spark UI stops being served when the application finishes. +This can sometimes be frustrating when debugging an application. +You can delay the stopping of the UI by setting `spark.ui.stopDelay` duration, e.g. "5m" for 5 minutes. +Note that this will cause the driver process to continue running. +You can force a delayed stop to proceed immediately on the "Jobs" page of the web UI, +or by sending + + +# Remote URLs + +Nomad uses [go-getter](https://github.com/hashicorp/go-getter) to download artifacts, +which allows you to embed checksums in HTTP/S URLs. +Using checksums is recommended, as it allows Nomad to both verify the integrity of a file, +and use the checksum as a cache key to avoid re-downloading unchanged files. +The checksums take the form of a query string parameter of the form `checksum=type:value`, +where `type` is a hash type and `value` is the checksum value. +See the [go-getter checksumming documentation](https://github.com/hashicorp/go-getter#checksumming) +for details. + + +# Nomad Job Customization + +By default, Nomad will start with a blank job, and starts adding to it. +When running in cluster mode, it will add a task group for the driver, +containing a task (with a "spark.nomad.role" = "driver" meta value) to run the driver. +The driver then adds a task group to run its executors. +The executor task group will contain a task (with a "spark.nomad.role" = "executor" meta value) to run the executor, +and if the shuffle service is enabled (as with [dynamic allocation](#dynamic-allocation-of-executors)), also a task (with a "spark.nomad.role" = "shuffle" meta value) to run the shuffle service. + +The following incomplete HCL job specification gives an idea of the structure of a Nomad job created by Spark. + +~~~ .hcl +job "structure" { + meta { + "spark.nomad.role" = "application" + } + + # A driver group is only added in cluster mode + group "driver" { + task "driver" { + meta { + "spark.nomad.role" = "driver" + } + } + } + + group "executors" { + count = 2 + + task "executor" { + meta { + "spark.nomad.role" = "executor" + } + } + + # shuffle service tasks are only added when enabled (as it must be when using dynamic allocation) + task "shuffle-service" { + meta { + "spark.nomad.role" = "shuffle" + } + } + } +} +~~~ + +There are two ways to customize the Nomad job, task groups and tasks that Spark creates. +You can provide a job template that Spark will use as a starting point for creating its Nomad job, allowing you to customize almost any aspect of the job. +You can also set Spark configuration properties to override e.g. how many resources Spark should reserve for its Nomad tasks. + +The order of precedence for settings is as follows: +1. Explicitly set configuration properties. +2. Settings in the job template if provided. +3. Default values of the configuration properties. + + +## Using a Job Template + +Rather than having Spark create a Nomad job from scratch to run your application, +you can set the `spark.nomad.job.template` configuration property (set by `spark-submit`'s `--nomad-template` flag) +to the path of the file containing a template job specification. +There are two important things to note here: +- The template must be in the format of a [JSON job specification](https://www.nomadproject.io/docs/http/json-jobs.html). + Nomad job specifications are normally [written in HCL](https://www.nomadproject.io/docs/job-specification/) and converted to JSON + by the `nomad` command-line tool. You can convert an HCL jobspec to JSON by running `nomad run -output `. +- `spark.nomad.job.template` should be set to a path on the submitting machine, *not* to a URL (even in cluster mode). + The template does not need to be accessible from driver or executors. + +Using a job template you can override Spark's default resource utilization, +add additional metadata or constraints, set environment variables, add sidecar tasks to the driver or executor task groups, +add additional task groups of your own, etc. + +The template does not need to be a complete Nomad jobspec, as Spark will add everything necessary to run your Spark application. +E.g., your template might set some job metadata, but not contain any task groups, +making it an incomplete Nomad jobspec but still a valid template to use with Spark. + +When setting properties in the job Spark creates from your template, the value precedence is as follows: +1. Values from Spark properties +2. Values in the job template +3. Default values for Spark configuration properties + +E.g. the job priority is controlled by the `spark.nomad.priority` configuration property, which has a default of 40. +If a value is specified for that property (e.g. on the commandline, in `spark-defaults.conf`, etc.), that value will be used. +Otherwise if the template contains a value for the priority, the template's value will be used. +If neither of these sources provides a value, the default of 40 will be used. + +Conceptually, this is how Spark uses the job template: + +1. Identify the template task group for Spark executors, of which there should be at most one, + as a group containing tasks with a "spark.nomad.role" meta value of "executor" or "shuffle". + If there is such a group, it is removed from the job and used as a template for executors. + +2. Identify the template task group for the Spark driver, of which there should be at most one, + as a group containing a task with a "spark.nomad.role" meta value of "driver". + If not running in cluster mode, this task group is removed from the job and discarded. + +3. Proceed as normal, but using the provided job, driver task group, and executor task group template + as the starting point for the task groups that are normally generated. + +Here's an example of a small HCL template that sets a metadata value on the job and an environment +variable for executors: + +~~~ .hcl +job "template" { + meta { + "foo" = "bar" + } + group "executor-group-name" { + task "executor-task-name" { + meta { + "spark.nomad.role" = "executor" + } + env { + BAZ = "something" + } + } + } +} +~~~ + +Note that this is only a partial jobspec and not fully runnable. But it is a valid template, and can be converted to JSON with `nomad run -output `. +The job name is always set at runtime, so the "template" name is just a syntactically necessary placeholder and will be overridden. +Also note that it is the "spark.nomad.role" = "executor" meta value on the driver task that tells Spark that the "driver-group-name" task group is the template for executors; +the names of the group and task can be whatever you like. + + +## Resource Allocation + +Resource allocation can be configured using a job template or through configuration properties. + +Configuring resources using a template would look something like this (this HCL syntax; see the section above on converting this to JSON): + +~~~ .hcl +job "template" { + group "group-name" { + task "task-name" { + meta { + "spark.nomad.role" = "role" # this would be "driver", "executor", or "shuffle", as appropriate + } + + resources { + cpu = 2000 + memory = 2048 + + network { + mbits = 100 + } + } + } + } +} +~~~ + +Resource-related configuration properties are covered below: + +### Memory + +The standard Spark memory properties will be propagated to Nomad to control task resource allocation: +`spark.driver.memory` (set by `spark-submit`'s `--driver-memory` flag) and +`spark.executor.memory` (set by `spark-submit`'s `--executor-memory` flag). +You can additionally specify `spark.nomad.shuffle.memory` to control how much memory Nomad allocates to shuffle service tasks. + +### CPU + +Spark sizes its thread pools and allocates tasks based on the number of CPU cores available. +Nomad manages CPU allocation in terms of processing speed rather than number of cores. + +When running Spark on Nomad, you can control how much CPU share Nomad will allocate to tasks using +the `spark.nomad.driver.cpu`, `spark.nomad.executor.cpu` and `spark.nomad.shuffle.cpu` properties +(or `spark-submit`'s `--driver-cpu` and `--executor-cpu` flags). + +When running on Nomad, executors will be configured to use one core by default, +meaning they will only pull a single 1-core task at a time. +You can setting the `spark.executor.cores` property (set by `spark-submit`'s `--executor-cores` flag) +to allow more tasks to be executed concurrently on a single executor. + +### Network + +Nomad doesn't restrict the network bandwidth of running tasks, +bit it does allocate a non-zero number of Mbit/s to each task and uses this when bin-packing task groups onto Nomad clients. +Spark defaults to requesting the minimum of 1 Mbit/s per task, but you can change this with the +`spark.nomad.driver.networkMBits`, `spark.nomad.executor.networkMBits`, and `spark.nomad.shuffle.networkMBits` properties. + +### Log Rotation + +Nomad performs log rotation on the `stdout` and `stderr` of its tasks. +You can configure the number number and size of log files it will keep for driver and executor task groups using +`spark.nomad.driver.logMaxFiles` and `spark.nomad.executor.logMaxFiles`. + + +# Logs + +Nomad clients collect the stderr and stdout of the tasks that they run, +and the nomad CLI or API can be used to inspect them, as documented in Nomad's documentation on +[Accessing Logs](https://www.nomadproject.io/docs/operating-a-job/accessing-logs.html). + +Links to the stderr and stdout of the executors tab of the Spark UI. +In cluster mode, the stderr and stdout of the driver application can be found there as well. + +The Log Shipper Pattern described in the "Accessing Logs" link above uses sidecar tasks to forward logs to a central location. +This can be done [using a job template](#using-a-job-template) along the following lines: + +~~~ .hcl +job "template" { + + group "driver" { + task "driver" { + meta { + "spark.nomad.role" = "driver" + } + } + task "log-forwarding-sidecar" { + # sidecar task definition here + } + } + + group "executor" { + task "executor" { + meta { + "spark.nomad.role" = "executor" + } + } + task "log-forwarding-sidecar" { + # sidecar task definition here + } + } + +} +~~~ + + +# Dynamic Allocation of Executors + +By default, the Spark application will use a fixed number of executors. +Setting `spark.dynamicAllocation` to `true` enables Spark to +add and remove executors during execution depending on the number of Spark tasks scheduled to run. +As described in [Dynamic Resource Allocation](http://spark.apache.org/docs/latest/job-scheduling.html#configuration-and-setup), +dynamic allocation requires that `spark.shuffle.service.enabled` be set to `true`. + +On Nomad, setting `spark.shuffle.service.enabled` to `true` adds an additional +shuffle serivce Nomad task to the executors' task group. This results in a +one-to-one mapping of executors to shuffle services. + +When the executor exits, the shuffle service continues running so that it can serve any results produced by the +executor. Note that due to the way resource allocation works in Nomad, +the resources allocated to the executor Nomad task aren't freed until the shuffle service +is also finished, meaning that they will remain allocated until the application has finished. +This may improve in the future. + + +# Python and R + +There is basic support for running Spark applications written in Python and R on Nomad, +including the `pyspark` and `sparkR` interactive modes. + +For example, running a python Spark application on Nomad: + +``` +bin/spark-submit \ + --master nomad \ + --distribution http://example.com/spark.tgz \ + examples/src/main/python/pi.py +``` + +For example, running a sparkR interactive mode with executors on Nomad: + +``` +bin/sparkR \ + --master nomad \ + --distribution http://example.com/spark.tgz +``` + +Note that the python or R runtime must be installed on the Nomad clients; +if these are only present on some client, you can use constraints (documented above) +to ensure your task groups run on these clients. + + +# Configuration + +Most of the configuration properties are the same for Spark on Nomad as for other deployment modes. +See the [configuration page](configuration.html) for more information on those. + +These are configuration properties that are specific to running Spark on Nomad. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.executor.instances2 + The number of executors for static allocation. With spark.dynamicAllocation.enabled, the initial set of executors will be at least this large. +
    spark.nomad.cluster.expectImmediateSchedulingfalse + When true, spark-submit will fail if Nomad isn't able to schedule the job to run right away +
    spark.nomad.cluster.monitorUntil + Specifies how long spark-submit should monitor a Spark application in cluster mode. submitted (the default) causes spark-submit to return as soon as the application has been submitted to the Nomad cluster. scheduled causes spark-submit to return once the Nomad job has been scheduled. complete causes spark-submit to tail the output from the driver process and return when the job has completed. +
    spark.nomad.datacenters + Comma-separated list of Nomad datacenters to use (defaults to the datacenter of the first Nomad server contacted) +
    spark.nomad.docker.email + Email address used when downloading the docker image specified by spark.nomad.dockerImage from the docker registry. (https://www.nomadproject.io/docs/drivers/docker.html#authentication) +
    spark.nomad.docker.password + Password used when downloading the docker image specified by spark.nomad.dockerImage from the docker registry. (https://www.nomadproject.io/docs/drivers/docker.html#authentication) +
    spark.nomad.docker.serverAddress + Server address (domain/IP without the protocol) used when downloading the docker image specified by spark.nomad.dockerImage from the docker registry. Docker Hub is used by default.(https://www.nomadproject.io/docs/drivers/docker.html#authentication) +
    spark.nomad.docker.username + Username used when downloading the docker image specified by spark.nomad.dockerImage from the docker registry. (https://www.nomadproject.io/docs/drivers/docker.html#authentication) +
    spark.nomad.dockerImage + A [docker image](https://www.nomadproject.io/docs/drivers/docker.html#image) to use to run Spark with Nomad's docker driver. When not specified, Nomad's exec driver will be used instead. +
    spark.nomad.driver.cpu1000 + How many MHz of CPU power Nomad should reserve for driver tasks +
    spark.nomad.driver.logMaxFileSize1m + Maximum size that Nomad should keep in log files from driver tasks +
    spark.nomad.driver.logMaxFiles5 + Number of log files Nomad should keep from driver tasks +
    spark.nomad.driver.networkMBits1 + The network bandwidth Nomad should allocate to driver tasks during bin packing +
    spark.nomad.driver.retryAttempts5 + The number of times Nomad should retry driver task groups if they fail +
    spark.nomad.driver.retryDelay15s + How long Nomad should wait before retrying driver task groups if they fail +
    spark.nomad.driver.retryInterval1d + Nomad's retry interval for driver task groups +
    spark.nomad.executor.cpu1000 + How many MHz of CPU power Nomad should reserve for executor tasks +
    spark.nomad.executor.logMaxFileSize1m + Maximum size that Nomad should keep in log files from executor tasks +
    spark.nomad.executor.logMaxFiles5 + Number of log files Nomad should keep from executor tasks +
    spark.nomad.executor.networkMBits1 + The network bandwidth Nomad should allocate to executor tasks during bin packing +
    spark.nomad.executor.retryAttempts5 + The number of times Nomad should retry executor task groups if they fail +
    spark.nomad.executor.retryDelay15s + How long Nomad should wait before retrying executor task groups if they fail +
    spark.nomad.executor.retryInterval1d + Nomad's retry interval for executor task groups +
    spark.nomad.job + The Nomad job name to use +
    spark.nomad.job.template + The path to a JSON file containing a Nomad job to use as a template +
    spark.nomad.priority + The priority of the Nomad job that runs the application or its executors +
    spark.nomad.region + The Nomad region to use (defaults to the region of the first Nomad server contacted) +
    spark.nomad.shuffle.cpu1000 + How many MHz of CPU power Nomad should reserve for shuffle service tasks +
    spark.nomad.shuffle.logMaxFileSize1m + Maximum size that Nomad should keep in log files from shuffle service tasks +
    spark.nomad.shuffle.logMaxFiles5 + Number of log files Nomad should keep from shuffle service tasks +
    spark.nomad.shuffle.memory256m + The amount of memory that Nomad should allocate for the shuffle service tasks +
    spark.nomad.shuffle.networkMBits1 + The network bandwidth Nomad should allocate to shuffle service tasks during bin packing +
    spark.nomad.sparkDistribution + The location of the spark distribution tgz file to use. +
    spark.nomad.tls.caCert + Path to a .pem file containing the certificate authority to validate the Nomad server's TLS certificate against +
    spark.nomad.tls.cert + Path to a .pem file containing the TLS certificate to present to the Nomad server +
    spark.nomad.tls.trustStorePassword + Path to a .pem file containing the private key corresponding to the certificate in spark.nomad.tls.cert +
    diff --git a/docs/security.md b/docs/security.md index 9eda42888637f..7a9b91309d998 100644 --- a/docs/security.md +++ b/docs/security.md @@ -6,7 +6,7 @@ title: Security Spark currently supports authentication via a shared secret. Authentication can be configured to be on via the `spark.authenticate` configuration parameter. This parameter controls whether the Spark communication protocols do authentication using the shared secret. This authentication is a basic handshake to make sure both sides have the same shared secret and are allowed to communicate. If the shared secret is not identical they will not be allowed to communicate. The shared secret is created as follows: -* For Spark on [YARN](running-on-yarn.html) deployments, configuring `spark.authenticate` to `true` will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. +* For Spark on [YARN](running-on-yarn.html) or [Nomad](running-on-nomad.html) deployments, configuring `spark.authenticate` to `true` will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. * For other types of Spark deployments, the Spark parameter `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. ## Web UI diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md index a5d36da5b6de9..2c6fa71fd2b2a 100644 --- a/docs/streaming-flume-integration.md +++ b/docs/streaming-flume-integration.md @@ -68,7 +68,7 @@ configuring Flume agents. Note that the hostname should be the same as the one used by the resource manager in the - cluster (Mesos, YARN or Spark Standalone), so that resource allocation can match the names and launch + cluster (Mesos, YARN, Nomad or Spark Standalone), so that resource allocation can match the names and launch the receiver in the right machine. 3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 6c0c3ebcaebf4..a02f16094d316 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -157,6 +157,7 @@ List buildClassPath(String appClassPath) throws IOException { "mllib", "repl", "resource-managers/mesos", + "resource-managers/nomad", "resource-managers/yarn", "sql/catalyst", "sql/core", diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java index 137ef74843da5..fdca66f84873b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java @@ -75,6 +75,11 @@ public List buildCommand(Map env) memKey = "SPARK_EXECUTOR_MEMORY"; extraClassPath = getenv("SPARK_EXECUTOR_CLASSPATH"); break; + case "org.apache.spark.executor.NomadExecutorBackend": + javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); + memKey = "SPARK_EXECUTOR_MEMORY"; + extraClassPath = getenv("SPARK_EXECUTOR_CLASSPATH"); + break; case "org.apache.spark.deploy.mesos.MesosClusterDispatcher": javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); break; diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 6767cc5079649..c8b70f313dc9d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -58,6 +58,8 @@ class SparkSubmitOptionParser { protected final String REPOSITORIES = "--repositories"; protected final String STATUS = "--status"; protected final String TOTAL_EXECUTOR_CORES = "--total-executor-cores"; + protected final String EXECUTOR_CORES = "--executor-cores"; + protected final String NUM_EXECUTORS = "--num-executors"; // Options that do not take arguments. protected final String HELP = "--help"; @@ -68,11 +70,18 @@ class SparkSubmitOptionParser { // Standalone-only options. + // Nomad-only options. + protected final String DISTRIBUTION = "--distribution"; + protected final String DOCKER_IMAGE = "--docker-image"; + protected final String DRIVER_CPU = "--driver-cpu"; + protected final String EXECUTOR_CPU = "--executor-cpu"; + protected final String MONITOR_UNTIL = "--monitor-until"; + protected final String NOMAD_DRY_RUN = "--nomad-dry-run"; + protected final String NOMAD_TEMPLATE = "--nomad-template"; + // YARN-only options. protected final String ARCHIVES = "--archives"; - protected final String EXECUTOR_CORES = "--executor-cores"; protected final String KEYTAB = "--keytab"; - protected final String NUM_EXECUTORS = "--num-executors"; protected final String PRINCIPAL = "--principal"; protected final String QUEUE = "--queue"; @@ -90,20 +99,26 @@ class SparkSubmitOptionParser { { ARCHIVES }, { CLASS }, { CONF, "-c" }, + { DISTRIBUTION }, { DEPLOY_MODE }, + { DOCKER_IMAGE }, { DRIVER_CLASS_PATH }, { DRIVER_CORES }, + { DRIVER_CPU }, { DRIVER_JAVA_OPTIONS }, { DRIVER_LIBRARY_PATH }, { DRIVER_MEMORY }, { EXECUTOR_CORES }, + { EXECUTOR_CPU }, { EXECUTOR_MEMORY }, { FILES }, { JARS }, { KEYTAB }, { KILL_SUBMISSION }, { MASTER }, + { MONITOR_UNTIL }, { NAME }, + { NOMAD_TEMPLATE }, { NUM_EXECUTORS }, { PACKAGES }, { PACKAGES_EXCLUDE }, @@ -121,6 +136,7 @@ class SparkSubmitOptionParser { * List of switches (command line options that do not take parameters) recognized by spark-submit. */ final String[][] switches = { + { NOMAD_DRY_RUN }, { HELP, "-h" }, { SUPERVISE }, { USAGE_ERROR }, diff --git a/pom.xml b/pom.xml index 0533a8dcf2e0a..a9beb73bcf721 100644 --- a/pom.xml +++ b/pom.xml @@ -157,7 +157,7 @@ 2.11.8 2.11 1.9.13 - 2.6.5 + 2.7.9 1.1.2.6 1.1.2 1.2.0-incubating @@ -2539,6 +2539,14 @@
    + + nomad + + resource-managers/nomad/test-apps + resource-managers/nomad + + + hive-thriftserver diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b5362ec1ae452..38d6cf2288306 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -56,9 +56,9 @@ object BuildCommons { "tags", "sketch" ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects - val optionallyEnabledProjects@Seq(mesos, yarn, sparkGangliaLgpl, + val optionallyEnabledProjects@Seq(mesos, nomad, yarn, sparkGangliaLgpl, streamingKinesisAsl, dockerIntegrationTests, hadoopCloud) = - Seq("mesos", "yarn", "ganglia-lgpl", "streaming-kinesis-asl", + Seq("mesos", "nomad", "yarn", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests", "hadoop-cloud").map(ProjectRef(buildLocation, _)) val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) = diff --git a/resource-managers/nomad/pom.xml b/resource-managers/nomad/pom.xml new file mode 100644 index 0000000000000..5f4a475a6bcd6 --- /dev/null +++ b/resource-managers/nomad/pom.xml @@ -0,0 +1,96 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-nomad_2.11 + jar + Spark Project Nomad + + nomad + + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + + com.google.guava + guava + 14.0.1 + compile + + + + com.hashicorp.nomad + nomad-scala-sdk_${scala.binary.version} + 0.0.3 + + + + org.apache.httpcomponents + httpclient + + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-nomad-test-apps_${scala.binary.version} + ${project.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + com.hashicorp.nomad + nomad-testkit + 0.0.3 + test + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + diff --git a/resource-managers/nomad/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/resource-managers/nomad/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager new file mode 100644 index 0000000000000..c999790a15ad4 --- /dev/null +++ b/resource-managers/nomad/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -0,0 +1 @@ +org.apache.spark.scheduler.cluster.nomad.NomadClusterManager diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/ApplicationRunCommand.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/ApplicationRunCommand.scala new file mode 100644 index 0000000000000..2c9a315c99891 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/ApplicationRunCommand.scala @@ -0,0 +1,26 @@ +/* + * 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.nomad + +import org.apache.spark.deploy.nomad.NomadClusterModeLauncher.PrimaryResource + +private[spark] case class ApplicationRunCommand( + primaryResource: PrimaryResource, + mainClass: String, + arguments: Seq[String] +) diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadClusterModeConf.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadClusterModeConf.scala new file mode 100644 index 0000000000000..f3b2bafb28333 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadClusterModeConf.scala @@ -0,0 +1,90 @@ +/* + * 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.nomad + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.nomad.NomadClusterModeLauncher._ +import org.apache.spark.internal.config._ +import org.apache.spark.scheduler.cluster.nomad._ +import org.apache.spark.util.Utils + +/** + * Configuration for submitting an application to nomad in cluster mode. + * + * In order to fail fast in the face of configuration errors, + * we extract all the configuration as soon as possible, including backend config. + * This allows us fail before we interact with Nomad at all, instead of at some later point. + */ +private[spark] case class NomadClusterModeConf( + backend: NomadClusterManagerConf, + expectImmediateScheduling: Boolean, + monitorUntil: Option[Milestone] +) + +private[spark] object NomadClusterModeConf { + + val EXPECT_IMMEDIATE_SCHEDULING = + ConfigBuilder("spark.nomad.cluster.expectImmediateScheduling") + .doc("When true, spark-submit will fail if Nomad isn't able to schedule the job to run " + + "right away") + .booleanConf + .createWithDefault(false) + + val MONITOR_UNTIL = + ConfigBuilder("spark.nomad.cluster.monitorUntil") + .doc("Specifies how long spark-submit should monitor a Spark application in cluster mode. " + + "`submitted` (the default) causes spark-submit to return as soon as the application has " + + "been submitted to the Nomad cluster. `scheduled` causes spark-submit to return once " + + "the Nomad job has been scheduled. `complete` causes spark-submit to tail the output " + + "from the driver process and return when the job has completed.") + .stringConf + .createOptional + + def apply(conf: SparkConf, command: ApplicationRunCommand): NomadClusterModeConf = { + + val backendConf = NomadClusterManagerConf(conf, Some(command)) + + // Fail fast if dynamic execution is enabled but the external shuffle service isn't + // (otherwise we would create a Nomad job to run the driver, and it would fail to start) + if (Utils.isDynamicAllocationEnabled(conf) && !conf.get(SHUFFLE_SERVICE_ENABLED)) { + throw new SparkException("Dynamic allocation of executors requires the external " + + "shuffle service. You may enable this through spark.shuffle.service.enabled.") + } + + val expectImmediateScheduling = conf.get(EXPECT_IMMEDIATE_SCHEDULING) + + NomadClusterModeConf( + backend = backendConf, + expectImmediateScheduling = expectImmediateScheduling, + monitorUntil = conf.get(MONITOR_UNTIL).map(_.toLowerCase() match { + case "submitted" if expectImmediateScheduling => + throw new IllegalArgumentException( + s"$EXPECT_IMMEDIATE_SCHEDULING is true but $MONITOR_UNTIL is `submitted`") + case "submitted" => Submitted + case "scheduled" => Scheduled + case "complete" => Complete + case value => + throw new IllegalArgumentException( + """spark.nomad.traceUntil must be "submitted", "scheduled" or "complete"""" + ) + }) + + ) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadClusterModeLauncher.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadClusterModeLauncher.scala new file mode 100644 index 0000000000000..8b51fca332e70 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadClusterModeLauncher.scala @@ -0,0 +1,218 @@ +/* + * 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.nomad + +import scala.annotation.tailrec +import scala.collection.mutable.ArrayBuffer + +import com.hashicorp.nomad.apimodel.Evaluation +import com.hashicorp.nomad.javasdk.WaitStrategy +import com.hashicorp.nomad.scalasdk.NomadScalaApi + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} +import org.apache.spark.nomad.{EvaluationUtils, JobUtils} +import org.apache.spark.scheduler.cluster.nomad.SparkNomadJobController + +/** + * Launches an application in a Nomad cluster. + * + * In cluster mode, a Nomad job is registered with a single task that runs the Nomad driver. + * The driver process then adds executor task groups to the job as necessary. + */ +private[spark] class NomadClusterModeLauncher( + conf: SparkConf, + clusterModeConf: NomadClusterModeConf +) extends Logging { + import org.apache.spark.deploy.nomad.NomadClusterModeLauncher._ + + private val launcherBackend = new LauncherBackend { + connect() + + override protected def onStopRequest(): Unit = {} + } + + logInfo(s"Will access Nomad API at ${clusterModeConf.backend.nomadApi.getAddress}") + private val nomad = NomadScalaApi(clusterModeConf.backend.nomadApi) + private val jobController = SparkNomadJobController.initialize(clusterModeConf.backend, nomad) + launcherBackend.setAppId(jobController.jobId) + + val waitForever = new WaitStrategy { + override def getWait: String = "10s" + } + + def submit(): Unit = { + try { + val evaluation = jobController.startDriver() + logInfo(s"Nomad job with driver task submitted") + reportOutcome(evaluation) + } catch { + case e: Throwable => + logError("Driver failure", e) + launcherBackend.setState(SparkAppHandle.State.FAILED) + throw e + } + } + + def reportOutcome(initialEvaluation: Evaluation): Unit = { + launcherBackend.setState(SparkAppHandle.State.SUBMITTED) + + val monitorUntil = clusterModeConf.monitorUntil.getOrElse { + if (launcherBackend.isConnected()) Complete + else if (clusterModeConf.expectImmediateScheduling) Scheduled + else Submitted + } + + if (monitorUntil == Scheduled || monitorUntil == Complete) { + val evaluation = new EvaluationUtils(nomad).traceUntilFullyScheduled( + initialEvaluation, + clusterModeConf.expectImmediateScheduling, + waitForever + ) + val jobUtils = new JobUtils(nomad) + monitorUntil match { + case Submitted => + // done + case Scheduled => + val (nodeId, sparkUiAddress) = + jobUtils.pollTaskGroupAllocation(evaluation.getJobId, "driver", waitForever) { + alloc => + JobUtils.extractPortAddress(alloc, "driver", "SparkUI") + .map(alloc.getNodeId -> _) + } + log.info( + s"The driver's Spark UI will be served on node $nodeId at http://$sparkUiAddress/" + ) + case Complete => + jobUtils.traceTask(evaluation.getJobId, "driver", "driver", waitForever) { + launcherBackend.setState(SparkAppHandle.State.RUNNING) + } + logInfo("Driver completed successfully") + } + launcherBackend.setState(SparkAppHandle.State.FINISHED) + } + } +} + + +private[spark] object NomadClusterModeLauncher extends Logging { + + private def usageException(message: String): Nothing = { + throw new IllegalArgumentException( + s"""Error: $message + |Usage: org.apache.spark.deploy.nomad.NomadClusterModeLauncher [option...] + |Options: + | --jar JAR_PATH Path to your application's JAR file (required in nomad-cluster + | mode) + | --class CLASS_NAME Name of your application's main class (required) + | --primary-py-file A main Python file + | --primary-r-file A main R file + | --arg ARG Argument to be passed to your application's main class. + | Multiple invocations are possible, each will be passed in order. + |""".stripMargin + ) + } + + sealed trait PrimaryResource { + val url: String + } + + case class PrimaryJar(url: String) extends PrimaryResource + + case class PrimaryPythonFile(url: String) extends PrimaryResource + + case class PrimaryRFile(url: String) extends PrimaryResource + + def parseArguments(args: Seq[String]): ApplicationRunCommand = { + var mainClass: Option[String] = None + var primaryResource: Option[PrimaryResource] = None + var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]() + + def setPrimaryResource(resource: PrimaryResource): Unit = primaryResource match { + case None => primaryResource = Some(resource) + case Some(_) => + usageException("Cannot specify multiple primary resources (jar, python or R file") + } + + @tailrec + def parseRemaining(args: Seq[String]): Unit = { + args match { + case "--jar" +: value +: tail => + setPrimaryResource(PrimaryJar(value)) + parseRemaining(tail) + + case "--primary-py-file" +: value +: tail => + setPrimaryResource(PrimaryPythonFile(value)) + parseRemaining(tail) + + case "--primary-r-file" +: value +: tail => + setPrimaryResource(PrimaryRFile(value)) + parseRemaining(tail) + + case "--class" +: value +: tail => + mainClass match { + case None => mainClass = Some(value) + case Some(_) => usageException("--class cannot be specified multiple times") + } + parseRemaining(tail) + + case ("--arg") +: value +: tail => + userArgs += value + parseRemaining(tail) + + case Nil => + + case _ => + usageException(s"Unknown/unsupported param $args") + } + } + + parseRemaining(args) + + ApplicationRunCommand( + primaryResource = primaryResource + .getOrElse(usageException("Missing primary resource (jar, python or R file)")), + mainClass = mainClass.getOrElse(usageException("Missing class")), + arguments = userArgs + ) + } + + sealed trait Milestone + + case object Submitted extends Milestone + + case object Scheduled extends Milestone + + case object Complete extends Milestone + + private val milestonesByName = + Seq(Submitted, Scheduled, Complete) + .map(m => m.toString.toLowerCase() -> m) + .toMap + + def main(argStrings: Array[String]) { + logInfo(s"Running ${getClass.getName.stripSuffix("$")}") + val appCommand = parseArguments(argStrings) + val conf = new SparkConf + val clusterModeConf = NomadClusterModeConf(conf, appCommand) + + val result = new NomadClusterModeLauncher(conf, clusterModeConf).submit() + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadDryRun.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadDryRun.scala new file mode 100644 index 0000000000000..731fee88bdb5e --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/deploy/nomad/NomadDryRun.scala @@ -0,0 +1,45 @@ +/* + * 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.nomad + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.nomad.NomadClusterModeLauncher.parseArguments +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.cluster.nomad.NomadClusterManagerConf +import org.apache.spark.scheduler.cluster.nomad.NomadClusterManagerConf.NewJob + +private[spark] object NomadDryRun extends Logging { + + def main(argStrings: Array[String]) { + + val conf = new SparkConf + + val backendConf = conf.get("spark.submit.deployMode") match { + case "cluster" => NomadClusterModeConf(conf, parseArguments(argStrings)).backend + case _ => NomadClusterManagerConf(conf, None) + } + + val NewJob(job) = backendConf.jobDescriptor + + logInfo("Outputting JSON dry-run jobspec") + // scalastyle:off + System.out.println(job) + // scalastyle:on + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/executor/NomadExecutorBackend.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/executor/NomadExecutorBackend.scala new file mode 100644 index 0000000000000..ec86b3f64a264 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/executor/NomadExecutorBackend.scala @@ -0,0 +1,36 @@ +/* + * 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.executor + +/** + * Wraps a CoarseGrainedExecutorBackend invocation to set the executor ID. + */ +object NomadExecutorBackend { + + def main(args: Array[String]): Unit = { + val allocId = sys.env("NOMAD_ALLOC_ID") + val timestamp = System.currentTimeMillis() + + CoarseGrainedExecutorBackend.main( + Array( + "--executor-id", s"$allocId-$timestamp" + ) ++ args + ) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/nomad/EvaluationUtils.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/nomad/EvaluationUtils.scala new file mode 100644 index 0000000000000..6b52da612b180 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/nomad/EvaluationUtils.scala @@ -0,0 +1,88 @@ +/* + * 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.nomad + +import scala.collection.JavaConverters._ + +import com.hashicorp.nomad.apimodel.Evaluation +import com.hashicorp.nomad.javasdk.{ServerQueryResponse, WaitStrategy} +import com.hashicorp.nomad.scalasdk.{NomadScalaApi, ScalaQueryOptions} + +import org.apache.spark.internal.Logging + +private[spark] class EvaluationUtils(api: NomadScalaApi) extends Logging { + + /** + * Traces the execution of an evaluation until it completes. + * + * @param evaluationId the ID of the evaluation to trace + * @return a future that is completed when the evaluation completes + */ + def traceEvaluation(evaluationId: String, waitStrategy: WaitStrategy): Evaluation = { + var lastStatus: Option[String] = None + api.evaluations.info(evaluationId, Some(ScalaQueryOptions( + waitStrategy = Some(waitStrategy), + repeatedPollPredicate = Some({ response: ServerQueryResponse[Evaluation] => + val status = response.getValue.getStatus + if (!lastStatus.contains(status)) { + log.info("Evaluation {} is {}", evaluationId: Any, status) + lastStatus = Some(status) + } + status == "complete" + }) + ))).getValue + } + + def traceUntilFullyScheduled( + evaluation: Evaluation, + expectImmediateScheduling: Boolean, + waitStrategy: WaitStrategy): Evaluation = { + + evaluation.getBlockedEval match { + case null | "" => evaluation + case blockedEvalId => + val unallocatedTaskGroups = + Option(evaluation.getFailedTgAllocs).map(_.asScala).getOrElse(Map.empty) + + val message = unallocatedTaskGroups.mkString( + s"Blocked evaluation $blockedEvalId created for failed allocations:\n ", + "\n ", + "") + + if (expectImmediateScheduling) { + logError(message) + sys.error(message) + } else { + logInfo(message) + traceUntilFullyScheduled(blockedEvalId, expectImmediateScheduling, waitStrategy) + } + } + } + + def traceUntilFullyScheduled( + evaluationId: String, + expectImmediateScheduling: Boolean, + waitStrategy: WaitStrategy): Evaluation = { + + traceUntilFullyScheduled( + traceEvaluation(evaluationId, waitStrategy), + expectImmediateScheduling, + waitStrategy) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/nomad/JobUtils.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/nomad/JobUtils.scala new file mode 100644 index 0000000000000..8ac2fdade8da8 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/nomad/JobUtils.scala @@ -0,0 +1,248 @@ +/* + * 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.nomad + +import java.io.InputStreamReader +import java.math.BigInteger +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.Date + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ + +import com.google.common.io.LineReader +import com.hashicorp.nomad.apimodel._ +import com.hashicorp.nomad.javasdk.{ServerQueryResponse, WaitStrategy} +import com.hashicorp.nomad.scalasdk._ +import org.apache.log4j.{Level, LogManager} + +import org.apache.spark.internal.Logging + +private[spark] class JobUtils(api: NomadScalaApi) extends Logging { + + LogManager.getLogger(logName).setLevel(Level.DEBUG) + + def pollForNextAllocation( + jobId: String, + taskGroupName: String, + previousAllocation: Option[Allocation], + waitStrategy: WaitStrategy): AllocationListStub = { + @tailrec + def poll(pollIndex: Option[BigInteger]): AllocationListStub = { + logDebug(s"Polling alloc for task group $taskGroupName in $jobId") + val response = api.jobs.allocations(jobId, Some(ScalaQueryOptions( + index = pollIndex, + waitStrategy = Some(waitStrategy) + ))) + val untracedAllocations = response.getValue + .filter(a => a.getTaskGroup == taskGroupName && + previousAllocation.forall(_.getCreateIndex.compareTo(a.getCreateIndex) < 0)) + + if (untracedAllocations.isEmpty) poll(Some(response.getIndex)) + else untracedAllocations.minBy(_.getCreateIndex) + } + + poll(None) + } + + def traceTask( + jobId: String, + taskGroupName: String, + taskName: String, + waitStrategy: WaitStrategy)(onStarted: => Unit): Allocation = { + + val allocation = pollForNextAllocation(jobId, taskGroupName, None, waitStrategy) + traceTaskInAllocation(taskName, allocation.getId, waitStrategy, None)(onStarted) + } + + def pollTaskGroupAllocation[A]( + jobId: String, + taskGroupName: String, + waitStrategy: WaitStrategy + )(poll: Allocation => Option[A]): A = { + + val allocation = pollForNextAllocation(jobId, taskGroupName, None, waitStrategy) + pollAllocation(allocation.getId, waitStrategy)(poll) + } + + def pollAllocation[A](allocationId: String, waitStrategy: WaitStrategy) + (poll: Allocation => Option[A]): A = { + + val response = api.allocations.info(allocationId, Some(ScalaQueryOptions( + waitStrategy = Some(waitStrategy), + repeatedPollPredicate = Some( + (r: ServerQueryResponse[Allocation]) => poll(r.getValue).isDefined + ) + ))) + poll(response.getValue).get + } + + // TODO: remove non-tail recursion + def traceTaskInAllocation( + taskName: String, + allocationId: String, + waitStrategy: WaitStrategy, + previousResponse: Option[ServerQueryResponse[Allocation]] + )(onStarted: => Unit): Allocation = { + logDebug(s"Tracing alloc $allocationId for $taskName") + val response = api.allocations.info(allocationId, Some(ScalaQueryOptions( + waitStrategy = Some(waitStrategy), + index = previousResponse.map(_.getIndex) + ))) + + def extractTaskState(a: ServerQueryResponse[Allocation]): Option[TaskState] = + Option(a.getValue.getTaskStates).flatMap(states => Option(states.get(taskName))) + + val allocation = response.getValue + + val newEvents = extractTaskState(response) + .fold(Seq.empty[TaskEvent]) { taskState => + val events = taskState.getEvents.asScala + previousResponse + .flatMap(extractTaskState) + .flatMap(_.getEvents.asScala.lastOption) + .fold(events)(lastSeen => events.dropWhile(_.getTime <= lastSeen.getTime)) + } + + newEvents.foreach { e => + val date = new Date(e.getTime / 1000) + + val messages = (e.getType match { + case "Terminated" => Seq(s"Exit status ${e.getExitCode}") + case _ => Nil + }) ++ Seq(e.getMessage).filter(_ != null).filter(_ != "") + + val reasons = Seq( + e.getKillReason, + e.getRestartReason, + e.getTaskSignalReason + ).filter(_ != null).filter(_ != "") + + val errors = Seq( + e.getDownloadError, + e.getDriverError, + e.getKillError, + e.getSetupError, + e.getValidationError, + e.getVaultError + ).filter(_ != null).filter(_ != "") + + val message = ( + s"$taskName ${e.getType}" +: (messages ++ reasons ++ errors) + ).mkString(" -- ") + + if (errors.nonEmpty || e.getFailsTask) { + logError(message) + } else { + logInfo(message) + } + + if (e.getFailsTask || e.getType == "Killed") { + throw new RuntimeException(message) + } + + if (e.getType == "Started") { + onStarted.getClass + val client = api.lookupClientApiByNodeId(allocation.getNodeId) + logInfo("Now we're streaming") + + def stream(logType: String): Unit = { + new Thread(s"$taskName-$logType") { + setDaemon(true) + + override def run(): Unit = { + try { + val stream = client.logs(allocationId, taskName, follow = true, logType) + try { + val reader = new LineReader(new InputStreamReader(stream, UTF_8)) + @tailrec def pumpRemainingLines(): Unit = + reader.readLine() match { + case null => + log.info(s"$logType closed") + case line => + log.info(s"$logType: $line") + pumpRemainingLines() + } + pumpRemainingLines() + } catch { + case e: Throwable => + logError(s"Error while streaming $logType: $e", e) + try stream.close() + catch { + case _: Throwable => + } + } + } catch { + case e: Throwable => logError(s"Can't stream $logType: $e", e) + } + } + }.start() + } + + stream("stdout") + stream("stderr") + } + } + + val clientStatus = allocation.getClientStatus + if (previousResponse.forall(_.getValue.getClientStatus != clientStatus)) { + log.info("Allocation {} has client status {}", allocationId: Any, clientStatus) + } + + JobUtils.extractPortAddress(allocation, taskName, "SparkUI").foreach { address => + if (!previousResponse.exists(p => + JobUtils.extractPortAddress(p.getValue, taskName, "SparkUI").contains(address) + )) { + log.info(s"The driver's Spark UI will be served at http://$address/") + } + } + + clientStatus match { + case "complete" => + if (extractTaskState(response).exists(_.getFailed)) { + sys.error("Completed with error") + } else { + allocation + } + case _ => + traceTaskInAllocation(taskName, allocationId, waitStrategy, Some(response))(onStarted) + } + } + +} + +private[spark] object JobUtils { + + def extractPortAddress(allocation: Allocation, task: String, label: String): Option[String] = { + allocation.getTaskResources.get(task) match { + case null => None + case taskResources => taskResources.getNetworks match { + case null => None + case networks => networks.asScala.flatMap { network => + network.getDynamicPorts match { + case null => Nil + case ports => ports.asScala + .find(_.getLabel == label) + .map(port => network.getIp + ":" + port.getValue) + } + }.headOption + } + } + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DriverTask.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DriverTask.scala new file mode 100644 index 0000000000000..e414dab4ecbaa --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DriverTask.scala @@ -0,0 +1,135 @@ +/* + * 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.nomad + +import java.net.URI +import java.util.Arrays.asList + +import com.hashicorp.nomad.apimodel.{Service, Task} +import org.apache.http.HttpHost + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.nomad.ApplicationRunCommand +import org.apache.spark.deploy.nomad.NomadClusterModeLauncher.{PrimaryJar, PrimaryPythonFile, PrimaryRFile} +import org.apache.spark.internal.config.{DRIVER_MEMORY, PY_FILES} +import org.apache.spark.scheduler.cluster.nomad.SparkNomadJob.JOB_TEMPLATE +import org.apache.spark.util.Utils + +private[spark] object DriverTask extends SparkNomadTaskType("driver", "driver", DRIVER_MEMORY) { + + private val PROPERTIES_NOT_TO_FORWARD = scala.collection.Set( + "spark.master", + "spark.driver.port", + "spark.blockManager.port", + "spark.ui.port", + JOB_TEMPLATE.key, + PY_FILES.key, + "spark.app.id", + "spark.app.name", + "spark.submit.deployMode", + "spark.driver.extraClassPath", + "spark.driver.extraJavaOptions") + + case class Parameters(command: ApplicationRunCommand, nomadUrl: Option[HttpHost]) + + def configure( + jobConf: SparkNomadJob.CommonConf, + conf: SparkConf, + task: Task, + parameters: Parameters + ): Unit = { + val sparkUIEnabled = conf.getBoolean("spark.ui.enabled", true) + + val driverPort = ConfigurablePort("driver") + val blockManagerPort = ConfigurablePort("blockManager") + val sparkUIPort = + if (sparkUIEnabled) Some(ConfigurablePort("SparkUI")) + else None + val ports = Seq(driverPort, blockManagerPort) ++ sparkUIPort + + super.configure(jobConf, conf, task, ports, "spark-submit") + + val additionalJarUrls = Utils.getUserJars(conf).map(asFileIn(jobConf, task)) + if (additionalJarUrls.nonEmpty) { + conf.set("spark.jars", additionalJarUrls.mkString(",")) + } + + conf.getOption("spark.files") + .map(_.split(",").filter(_.nonEmpty)) + .filter(_.nonEmpty) + .foreach { files => + conf.set("spark.files", files.map(asFileIn(jobConf, task)).mkString(",")) + } + + val driverConf: Seq[(String, String)] = { + + val explicitConf = Seq( + "spark.app.id" -> jobConf.appId, + "spark.app.name" -> jobConf.appName, + "spark.driver.port" -> driverPort.placeholder, + "spark.blockManager.port" -> blockManagerPort.placeholder + ) ++ sparkUIPort.map("spark.ui.port" -> _.placeholder) + + val forwardedConf = conf.getAll + .filterNot { case (name, _) => PROPERTIES_NOT_TO_FORWARD.contains(name) } + + explicitConf ++ forwardedConf + } + + val command = parameters.command + val submitOptions: Seq[String] = Seq( + "--master=" + parameters.nomadUrl.fold("nomad")("nomad:" + _), + "--driver-class-path=" + + (additionalJarUrls ++ conf.getOption("spark.driver.extraClassPath")) + .map(j => new URI(j).getPath).mkString(":") + + ) ++ driverConf.map { case (name, value) => s"--conf=$name=$value" } + + val primaryResourceUrl = asFileIn(jobConf, task)(command.primaryResource.url) + val primaryResourceArgs: Seq[String] = command.primaryResource match { + case PrimaryJar(_) => + Seq(s"--class=${command.mainClass}", primaryResourceUrl) + case PrimaryPythonFile(_) => + val pythonOptions = conf.get(org.apache.spark.internal.config.PY_FILES) match { + case Nil => Nil + case pyFiles => Seq(pyFiles.map(asFileIn(jobConf, task)) + .mkString("--py-files=", ",", "")) + } + pythonOptions :+ primaryResourceUrl + case PrimaryRFile(_) => + Seq(primaryResourceUrl) + } + + appendArguments(task, submitOptions ++ primaryResourceArgs ++ command.arguments) + + sparkUIPort.foreach { port => + task.addServices( + new Service() + .setName(jobConf.appId + .toLowerCase + .replaceAll("[^a-z0-9]+", "-") + .replaceAll("^-?([a-z0-9-]{62}[a-z0-9]?).*", "$1") + ) + .setPortLabel(port.label) + .setTags(asList("SparkUI")) + ) + } + + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DriverTaskGroup.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DriverTaskGroup.scala new file mode 100644 index 0000000000000..7ea8e7a97c2ca --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DriverTaskGroup.scala @@ -0,0 +1,36 @@ +/* + * 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.nomad + +import com.hashicorp.nomad.apimodel.TaskGroup + +import org.apache.spark.SparkConf + +private[spark] object DriverTaskGroup extends SparkNomadTaskGroupType("driver", DriverTask) { + + def configure( + jobConf: SparkNomadJob.CommonConf, + conf: SparkConf, + group: TaskGroup, + parameters: DriverTask.Parameters + ): Unit = { + configureCommonSettings(jobConf, conf, group) + DriverTask.configure(jobConf, conf, findOrAdd(group, DriverTask), parameters) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ExecutorTask.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ExecutorTask.scala new file mode 100644 index 0000000000000..3375e5114ad56 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ExecutorTask.scala @@ -0,0 +1,97 @@ +/* + * 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.nomad + +import com.hashicorp.nomad.apimodel.Task + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.config.EXECUTOR_MEMORY + +private[spark] object ExecutorTask + extends SparkNomadTaskType("executor", "executor", EXECUTOR_MEMORY) { + + val LOG_KEY_FOR_ALLOC_ID = "nomad_alloc_id" + + private val PROPERTIES_NOT_TO_FORWARD_TO_EXECUTOR = scala.collection.Set( + "spark.driver.port", + "spark.blockManager.port", + "spark.ui.port") + + private val executorPort = ConfigurablePort("executor") + + def configure( + jobConf: SparkNomadJob.CommonConf, + conf: SparkConf, + task: Task, + shuffleServicePortPlaceholder: Option[String], + reconfiguring: Boolean + ): Task = { + + val cores = conf.getInt("spark.executor.cores", 1) + + val blockManagerPort = ConfigurablePort("blockManager") + + super.configure(jobConf, conf, task, Seq(executorPort, blockManagerPort), "spark-class") + + appendArguments(task, Seq( + "org.apache.spark.executor.NomadExecutorBackend", + "--hostname", executorPort.ipPlaceholder, + "--app-id", jobConf.appId, + "--cores", cores.toString + ), removeOld = reconfiguring) + + conf.getExecutorEnv.foreach((task.addEnv _).tupled) + + val executorConf: Seq[(String, String)] = { + + val explicitConf = Seq( + "spark.executor.port" -> executorPort.placeholder, + "spark.blockManager.port" -> blockManagerPort.placeholder + ) ++ shuffleServicePortPlaceholder.map("spark.shuffle.service.port" -> _) + + val forwardedConf = conf.getAll + .filter { case (name, _) => + (SparkConf.isExecutorStartupConf(name) || name == SecurityManager.SPARK_AUTH_SECRET_CONF + ) && !PROPERTIES_NOT_TO_FORWARD_TO_EXECUTOR.contains(name) + } + + explicitConf ++ forwardedConf + } + + task.addEnv("SPARK_EXECUTOR_OPTS", + executorConf.map { case (k, v) => s"-D$k=$v" + .replaceAllLiterally("\\", "\\\\") + .replaceAllLiterally("\"", "\\\"") + }.map('"' + _ + '"').mkString(" ")) + + task.addEnv("SPARK_EXECUTOR_MEMORY", jvmMemory(conf, task)) + + task.addEnv("SPARK_LOCAL_DIRS", "${NOMAD_ALLOC_DIR}") + + // Have the executor give its allocation ID as its log URL + // The driver will lookup the actual log URLs + task.addEnv("SPARK_LOG_URL_" + LOG_KEY_FOR_ALLOC_ID, "${NOMAD_ALLOC_ID}") + + task + } + + def addDriverUrlArguments(task: Task, driverUrl: String): Unit = { + appendArguments(task, Seq("--driver-url", driverUrl)) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ExecutorTaskGroup.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ExecutorTaskGroup.scala new file mode 100644 index 0000000000000..f968597f5acba --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ExecutorTaskGroup.scala @@ -0,0 +1,65 @@ +/* + * 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.nomad + +import com.hashicorp.nomad.apimodel.TaskGroup + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED +import org.apache.spark.scheduler.cluster.nomad.SparkNomadJob.CommonConf + +private[spark] object ExecutorTaskGroup + extends SparkNomadTaskGroupType("executor", ExecutorTask, ShuffleServiceTask) { + + def configure( + jobConf: SparkNomadJob.CommonConf, + conf: SparkConf, + group: TaskGroup, + reconfiguring: Boolean + ): Unit = { + configureCommonSettings(jobConf, conf, group) + + group.setCount(0) + + val shuffleServicePortPlaceholder = + if (conf.get(SHUFFLE_SERVICE_ENABLED)) { + val task = findOrAdd(group, ShuffleServiceTask) + Some(ShuffleServiceTask.configure(jobConf, conf, task, reconfiguring)) + } else { + find(group, ShuffleServiceTask).foreach(group.getTasks.remove) + None + } + + ExecutorTask.configure( + jobConf, conf, findOrAdd(group, ExecutorTask), shuffleServicePortPlaceholder, reconfiguring) + } + + def initialize( + conf: SparkConf, + group: TaskGroup, + driverUrl: String, + initialExecutors: Int + ): Unit = { + val task = find(group, ExecutorTask).get + configure(CommonConf(conf), conf, group, reconfiguring = true) + ExecutorTask.addDriverUrlArguments(task, driverUrl) + + group.setCount(initialExecutors) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterManager.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterManager.scala new file mode 100644 index 0000000000000..0a16435149d3a --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterManager.scala @@ -0,0 +1,64 @@ +/* + * 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.nomad + +import com.hashicorp.nomad.scalasdk.NomadScalaApi + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} + +/** + * A cluster manager that enables Spark applications to spawn executors on a Nomad cluster. + * + * The main implementation is in [[NomadClusterSchedulerBackend]]. + */ +private[spark] class NomadClusterManager extends ExternalClusterManager with Logging { + + override def canCreate(masterURL: String): Boolean = { + masterURL.startsWith("nomad") + } + + override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + new TaskSchedulerImpl(sc) + } + + override def createSchedulerBackend( + sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = { + + val clusterManagerConf = NomadClusterManagerConf(sc.conf, None) + + logInfo(s"Will access Nomad API at ${clusterManagerConf.nomadApi.getAddress}") + val api = NomadScalaApi(clusterManagerConf.nomadApi) + + val jobController = SparkNomadJobController.initialize(clusterManagerConf, api) + + new NomadClusterSchedulerBackend( + scheduler.asInstanceOf[TaskSchedulerImpl], + sc, + jobController, + clusterManagerConf.staticExecutorInstances) + } + + override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterManagerConf.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterManagerConf.scala new file mode 100644 index 0000000000000..352eb22fdbfb3 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterManagerConf.scala @@ -0,0 +1,154 @@ +/* + * 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.nomad + +import com.hashicorp.nomad.apimodel.Job +import com.hashicorp.nomad.javasdk.NomadApiConfiguration +import com.hashicorp.nomad.javasdk.NomadApiConfiguration.nomadAddressAsHttpHost +import org.apache.http.HttpHost + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.nomad.ApplicationRunCommand +import org.apache.spark.internal.config._ +import org.apache.spark.scheduler.cluster.nomad.NomadClusterManagerConf._ + +/** + * Configuration for running a Spark application on a Nomad cluster. + * + * This contains all the configuration used by [[NomadClusterSchedulerBackend]], + * other than configuration that is simply passed through to executors. + * + * By extracting all of the configuration as early as possible, + * we can fail fast in the face of configuration errors, + * rather than encountering them after we've already started interacting with Nomad. + */ +private[spark] case class NomadClusterManagerConf( + jobDescriptor: JobDescriptor, + nomadApi: NomadApiConfiguration, + staticExecutorInstances: Option[Int] +) + +private[spark] object NomadClusterManagerConf { + + val TLS_CA_CERT = + ConfigBuilder("spark.nomad.tls.caCert") + .doc("Path to a .pem file containing the certificate authority to validate the Nomad " + + "server's TLS certificate against") + .stringConf + .createOptional + + val TLS_CERT = + ConfigBuilder("spark.nomad.tls.cert") + .doc("Path to a .pem file containing the TLS certificate to present to the Nomad server") + .stringConf + .createOptional + + val TLS_KEY = + ConfigBuilder("spark.nomad.tls.trustStorePassword") + .doc("Path to a .pem file containing the private key corresponding to the certificate in " + + TLS_CERT.key) + .stringConf + .createOptional + + val DEFAULT_EXECUTOR_INSTANCES = 2 + + sealed trait JobDescriptor { + def id: String + def region: Option[String] + } + + case class ExistingJob( + override val id: String, + override val region: Option[String] + ) extends JobDescriptor + + case class NewJob(job: Job) extends JobDescriptor { + override def id: String = job.getId + override def region: Option[String] = Option(job.getRegion) + } + + case class KeyPair(certFile: String, keyFile: String) + + object KeyPair { + def apply(conf: SparkConf, + certEntry: ConfigEntry[Option[String]], + keyEntry: ConfigEntry[Option[String]]): Option[KeyPair] = + (conf.get(certEntry), conf.get(keyEntry)) match { + case (Some(cert), Some(file)) => Some(KeyPair(cert, file)) + case (None, None) => None + case _ => throw new SparkException( + s"You can either provide both ${certEntry.key} and ${keyEntry.key}, " + + "or neither of them, but you can't provide one an not the other." + ) + } + } + + def extractNomadUrl(conf: SparkConf): Option[HttpHost] = (conf.get("spark.master") match { + case "nomad" => sys.env.get("NOMAD_ADDR").map(_.trim).filter(_.nonEmpty) + case master if master.startsWith("nomad:") => Some(master.stripPrefix("nomad:")) + case invalid => throw new SparkException( + "Nomad master can either be \"nomad:\" followed by an explicit HTTP or HTTPS URL " + + "(e.g. nomad:http://nomad.example.com), or simply \"nomad\" which signals that " + + "the NOMAD_ADDR environment variable should be used if set and falls back to " + + "http://localhost:4646" + ) + }).map(nomadAddressAsHttpHost) + + def extractApiConf( + nomadUrl: Option[HttpHost], + region: Option[String], + conf: SparkConf + ): NomadApiConfiguration = { + val builder = new NomadApiConfiguration.Builder() + + builder.setFromEnvironmentVariables(System.getenv()) + + nomadUrl.foreach(address => builder.setAddress(address)) + + conf.get(TLS_CA_CERT).foreach(builder.setTlsCaFile) + KeyPair.apply(conf, TLS_CERT, TLS_KEY) + .foreach(p => builder.setTlsCertAndKeyFiles(p.certFile, p.keyFile)) + + region.foreach(builder.setRegion) + + builder.build() + } + + def apply( + conf: SparkConf, + command: Option[ApplicationRunCommand] + ): NomadClusterManagerConf = { + + val nomadUrl = extractNomadUrl(conf) + + val jobDescriptor = + if (conf.getBoolean(SparkNomadJob.SPARK_NOMAD_CLUSTER_MODE, defaultValue = false)) { + val jobConf = SparkNomadJob.CommonConf(conf) + ExistingJob(jobConf.appId, conf.get(SparkNomadJob.REGION)) + } else { + NewJob(SparkNomadJob(conf, nomadUrl, command)) + } + + NomadClusterManagerConf( + jobDescriptor = jobDescriptor, + nomadApi = extractApiConf(nomadUrl, jobDescriptor.region, conf), + staticExecutorInstances = conf.get(EXECUTOR_INSTANCES) + ) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterSchedulerBackend.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterSchedulerBackend.scala new file mode 100644 index 0000000000000..ceeae5e100a9d --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterSchedulerBackend.scala @@ -0,0 +1,189 @@ +/* + * 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.nomad + +import scala.concurrent.Future + +import org.apache.spark.SparkContext +import org.apache.spark.internal.config._ +import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} +import org.apache.spark.rpc.{RpcCallContext, RpcEndpointAddress, RpcEnv} +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.scheduler.cluster._ +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RequestExecutors} +import org.apache.spark.util.{ShutdownHookManager, Utils} + +/** + * A scheduler backend that runs Spark executors on a Nomad cluster. + * + * The executors will each run in their own task group within a single Nomad job. + * If the Spark application was submitted in cluster mode, the executors will be launched in the + * same Nomad job as the application driver. + * + * When dynamic resource allocation is enabled, each executor task group will have a shuffle-server + * task in addition to the executor task. + */ +private[spark] class NomadClusterSchedulerBackend( + scheduler: TaskSchedulerImpl, + sc: SparkContext, + jobController: SparkNomadJobController, + staticExecutorInstances: Option[Int] + ) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { + + private class NomadLauncherBackend extends LauncherBackend { + connect() + setAppId(applicationId()) + setState(SparkAppHandle.State.RUNNING) + + private val failureReportingShutdownHook = + ShutdownHookManager.addShutdownHook { () => + setState(SparkAppHandle.State.FAILED) + } + + def setFinalState(finalState: SparkAppHandle.State): Unit = + try setState(finalState) + finally { + ShutdownHookManager.removeShutdownHook(failureReportingShutdownHook) + close() + } + + override protected def onStopRequest(): Unit = + stop(SparkAppHandle.State.KILLED) + } + + private val launcherBackend = new NomadLauncherBackend + + override val minRegisteredRatio: Double = + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) 0.8 + else super.minRegisteredRatio + + private lazy val staticExecutorsToRequest: Option[Int] = + if (sc.executorAllocationManager.isDefined) None + else Some(staticExecutorInstances.getOrElse { + logWarning(s"Defaulting to ${NomadClusterManagerConf.DEFAULT_EXECUTOR_INSTANCES} executors." + + s" Consider setting ${EXECUTOR_INSTANCES.key} or enabling dynamic allocation.") + NomadClusterManagerConf.DEFAULT_EXECUTOR_INSTANCES + }) + + override def sufficientResourcesRegistered(): Boolean = { + val initiallyExpectedExecutors = staticExecutorsToRequest + .getOrElse(Utils.getDynamicAllocationInitialExecutors(conf)) + + totalRegisteredExecutors.get() >= initiallyExpectedExecutors * minRegisteredRatio + } + + override def applicationId(): String = jobController.jobId + + override def getDriverLogUrls: Option[Map[String, String]] = { + jobController.fetchDriverLogUrls(conf) + } + + override def stopExecutors(): Unit = { + jobController.setExecutorCount(0) + } + + override def stop(): Unit = { + stop(SparkAppHandle.State.FINISHED) + } + + def stop(finalState: SparkAppHandle.State): Unit = { + try { + try super.stop() + finally { + try jobController.setExecutorCount(0) + finally jobController.close() + } + } finally { + launcherBackend.setFinalState(finalState) + } + } + + /** + * Extend DriverEndpoint with Nomad-specific functionality. + */ + protected class NomadDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) + extends DriverEndpoint(rpcEnv, sparkProperties) { + + override def onStart(): Unit = { + + val driverUrl = RpcEndpointAddress( + conf.get("spark.driver.host"), + conf.get("spark.driver.port").toInt, + CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + + jobController.initialiseExecutors(conf, driverUrl, staticExecutorsToRequest.getOrElse(0)) + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = ({ + + /** + * Request additional executors as required. + * + * Doing this here instead of directly in createDriverEndpoint allows us to serialise all + * manipulations of the Nomad job. + */ + case RequestExecutors(total, _, _, _) => + jobController.setExecutorCount(total) + context.reply(true) + + /** + * Intercept registration messages from executors, resolve their log URLs, + * then relay a copy of the registration message with resolved log URLs to the parent class. + */ + case message@RegisterExecutor(executorId, _, _, _, logUrls) => + super.receiveAndReply(context)( + try message.copy(logUrls = jobController.resolveExecutorLogUrls(logUrls)) + catch { case e: Exception => + logWarning(s"Failed to lookup log URLs for executor $executorId", e) + message + } + ) + + }: PartialFunction[Any, Unit]) + .orElse(super.receiveAndReply(context)) + } + + /** + * Create a NomadDriverEndpoint. + */ + override protected def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { + new NomadDriverEndpoint(rpcEnv, properties) + } + + /** + * Add executor task groups as needed to meet the desired number. + * + * If the request total is lower, we don't need to do anything, + * as extra executors will eventually result in calls to [[doKillExecutors()]]. + */ + override protected def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = + driverEndpoint.ask[Boolean](RequestExecutors(requestedTotal, 0, Map.empty, Set.empty)) + + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { + logWarning( + executorIds.mkString( + s"Ignoring request to kill ${executorIds.size} executor(s): ", + ",", + " (not yet implemented for Nomad)" + ) + ) + super.doKillExecutors(executorIds) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadJobManipulator.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadJobManipulator.scala new file mode 100644 index 0000000000000..d111bf01df8a9 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/NomadJobManipulator.scala @@ -0,0 +1,113 @@ +/* + * 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.nomad + +import scala.annotation.tailrec + +import com.hashicorp.nomad.apimodel.{Evaluation, Job, Node} +import com.hashicorp.nomad.javasdk.{ErrorResponseException, WaitStrategy} +import com.hashicorp.nomad.scalasdk.NomadScalaApi + +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.cluster.nomad.NomadClusterManagerConf.{ExistingJob, JobDescriptor, NewJob} + +/** + * Convenience interface for manipulating a Nomad job. + */ +private[spark] class NomadJobManipulator(val nomad: NomadScalaApi, private var job: Job) + extends Logging { + + def jobId: String = job.getId + + def create(): Evaluation = { + register() + } + + @tailrec + final def updateJob(startIfNotYetRunning: Boolean)(modify: Job => Unit): Unit = { + modify(job) + logDebug("Trying to update to " + job) + try { + register() + } catch { + case e: ErrorResponseException => + log.warn(s"Updating job and retrying modification after error: $e") + val response = nomad.jobs.info(jobId) + job = response.getValue + updateJob(startIfNotYetRunning)(modify) + } + } + + def fetchLogUrlsForTask(allocId: String, task: String): Map[String, String] = { + val allocation = nomad.allocations.info(allocId).getValue + val node = nomad.nodes.info(allocation.getNodeId).getValue + Map( + "stdout" -> logUrl(node, allocId, task, "stdout"), + "stderr" -> logUrl(node, allocId, task, "stderr") + ) + } + + def logUrl(node: Node, allocId: String, task: String, log: String): String = { + val baseUrl = nomadHttpBaseUrl(node) + s"$baseUrl/v1/client/fs/logs/$allocId?follow=true&plain=true&task=$task&type=$log" + } + + def close(): Unit = { + logInfo(s"Closing Nomad API") + try nomad.close() + finally logInfo(s"Nomad API closed") + } + + protected def register(): Evaluation = { + val oldIndex = job.getJobModifyIndex + val evaluation = + nomad.evaluations.pollForCompletion( + nomad.jobs.register(job, modifyIndex = Some(oldIndex)), + WaitStrategy.WAIT_INDEFINITELY + ).getValue + val newIndex = evaluation.getJobModifyIndex + log.info(s"Registered Nomad job $jobId (job modify index $oldIndex -> $newIndex)") + job.setJobModifyIndex(newIndex) + evaluation + } + + private[this] def nomadHttpBaseUrl(node: Node): String = + (if (node.getTlsEnabled) "https://" else "http://") + node.getHttpAddr + +} + +private[spark] object NomadJobManipulator extends Logging { + + def fetchOrCreateJob(nomad: NomadScalaApi, jobDescriptor: JobDescriptor): NomadJobManipulator = { + + val job = jobDescriptor match { + + case ExistingJob(jobId, _) => + logInfo(s"Fetching current state of existing Nomad job $jobId") + val response = nomad.jobs.info(jobId) + logDebug(s"State: ${response.getRawEntity}") + response.getValue + + case NewJob(job) => + job + } + + new NomadJobManipulator(nomad, job) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ShuffleServiceTask.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ShuffleServiceTask.scala new file mode 100644 index 0000000000000..6ad1dcdcb5baf --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ShuffleServiceTask.scala @@ -0,0 +1,56 @@ +/* + * 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.nomad + +import com.hashicorp.nomad.apimodel.Task + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.network.util.ByteUnit + +private[spark] object ShuffleServiceTask + extends SparkNomadTaskType("shuffle", "shuffle service", + ConfigBuilder("spark.nomad.shuffle.memory") + .doc("The amount of memory that Nomad should allocate for the shuffle service tasks") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("256m") + ) { + + private val port = ConfigurablePort("shuffleService") + + def configure( + jobConf: SparkNomadJob.CommonConf, + conf: SparkConf, + task: Task, + reconfiguring: Boolean + ): String = { + super.configure(jobConf, conf, task, Seq(port), "spark-class") + + if (!reconfiguring) { + task + .addEnv("SPARK_SHUFFLE_OPTS", s"-Dspark.shuffle.service.port=$port") + .addEnv("SPARK_DAEMON_MEMORY", jvmMemory(conf, task)) + .addEnv("SPARK_LOCAL_DIRS", "${NOMAD_ALLOC_DIR}") + + appendArguments(task, Seq("org.apache.spark.deploy.ExternalShuffleService", "1")) + } + + port.placeholderInSiblingTasks(task) + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadJob.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadJob.scala new file mode 100644 index 0000000000000..2c03ad66e9291 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadJob.scala @@ -0,0 +1,305 @@ +/* + * 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.nomad + +import java.io.File +import java.math.BigInteger +import java.net.URI +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.Date + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import com.hashicorp.nomad.apimodel.{Job, Task, TaskGroup} +import com.hashicorp.nomad.javasdk.NomadJson +import org.apache.commons.lang3.time.DateFormatUtils.formatUTC +import org.apache.http.HttpHost + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.nomad.ApplicationRunCommand +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{ConfigBuilder, ConfigEntry, OptionalConfigEntry} + +private[spark] object SparkNomadJob extends Logging { + + val REGION = + ConfigBuilder("spark.nomad.region") + .doc("The Nomad region to use (defaults to the region of the first Nomad server contacted)") + .stringConf + .createOptional + + val DATACENTERS = + ConfigBuilder("spark.nomad.datacenters") + .doc("Comma-separated list of Nomad datacenters to use" + + " (defaults to the datacenter of the first Nomad server contacted)") + .stringConf + .createWithDefault("dc1") + + val NOMAD_JOB_PRIORITY = + ConfigBuilder("spark.nomad.priority") + .doc("The priority of the Nomad job that runs the application or its executors") + .intConf + .createWithDefault(40) + + val JOB_TEMPLATE = + ConfigBuilder("spark.nomad.job.template") + .doc("The path to a JSON file containing a Nomad job to use as a template") + .stringConf + .createOptional + + val DOCKER_IMAGE = + ConfigBuilder("spark.nomad.dockerImage") + .doc("A [docker image](https://www.nomadproject.io/docs/drivers/docker.html#image) " + + "to use to run Spark with Nomad's `docker` driver. " + + "When not specified, Nomad's `exec` driver will be used instead.") + .stringConf + .createOptional + + val DOCKER_AUTH_USERNAME = + ConfigBuilder("spark.nomad.docker.username") + .doc(s"Username used when downloading the docker image specified by `${DOCKER_IMAGE.key}` " + + "from the docker registry. " + + "(https://www.nomadproject.io/docs/drivers/docker.html#authentication) ") + .stringConf + .createOptional + + val DOCKER_AUTH_PASSWORD = + ConfigBuilder("spark.nomad.docker.password") + .doc(s"Password used when downloading the docker image specified by `${DOCKER_IMAGE.key}` " + + "from the docker registry. " + + "(https://www.nomadproject.io/docs/drivers/docker.html#authentication) ") + .stringConf + .createOptional + + val DOCKER_AUTH_EMAIL = + ConfigBuilder("spark.nomad.docker.email") + .doc("Email address used when downloading the docker image specified by " + + s"`${DOCKER_IMAGE.key}` from the docker registry. " + + "(https://www.nomadproject.io/docs/drivers/docker.html#authentication) ") + .stringConf + .createOptional + + val DOCKER_AUTH_SERVER_ADDRESS = + ConfigBuilder("spark.nomad.docker.serverAddress") + .doc("Server address (domain/IP without the protocol) used when downloading the docker " + + s"image specified by `${DOCKER_IMAGE.key}` from the docker registry. " + + "Docker Hub is used by default." + + "(https://www.nomadproject.io/docs/drivers/docker.html#authentication) ") + .stringConf + .createOptional + + val SPARK_DISTRIBUTION = + ConfigBuilder("spark.nomad.sparkDistribution") + .doc("The location of the spark distribution tgz file to use. " + + "If this isn't set, commands for all spark tasks must be explicitly set in the jobspec " + + s"at `${SparkNomadJob.JOB_TEMPLATE.key}`") + .stringConf + .createOptional + + val UTC_TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'" + + val SPARK_NOMAD_CLUSTER_MODE = "spark.nomad.clusterMode" + + case class CommonConf( + appId: String, + appName: String, + dockerImage: Option[String], + dockerAuth: Option[Map[String, String]], + sparkDistribution: Option[URI] + ) + + object CommonConf { + def apply(conf: SparkConf): CommonConf = { + val appName = conf.get("spark.app.name") + CommonConf( + appName = appName, + appId = conf.getOption("spark.app.id") match { + case None => s"${appName.replaceAll(" ", "_")}-$utcTimestamp" + case Some(id) => + if (id.contains(" ")) { + throw new SparkException( + "spark.app.id cannot contain a space when using a Nomad master") + } + id + }, + dockerImage = conf.get(DOCKER_IMAGE), + dockerAuth = { + val entries = Seq( + conf.get(DOCKER_AUTH_USERNAME).map("username" -> _), + conf.get(DOCKER_AUTH_PASSWORD).map("Password" -> _), + conf.get(DOCKER_AUTH_EMAIL).map("email" -> _), + conf.get(DOCKER_AUTH_SERVER_ADDRESS).map("server_address" -> _) + ).flatten + if (entries.nonEmpty) Some(entries.toMap) else None + }, + sparkDistribution = conf.get(SPARK_DISTRIBUTION).map(new URI(_)) + ) + } + } + + def apply( + conf: SparkConf, + nomadUrl: Option[HttpHost], + driverCommand: Option[ApplicationRunCommand] + ): Job = { + + val job = conf.get(JOB_TEMPLATE) match { + case None => + logInfo("Creating job without a template") + new Job() + case Some(file) => + logInfo("Creating job from provided template") + val job = NomadJson.readJobSpec(Files.toString(new File(file), UTF_8)) + if (job == null) { + throw new SparkException(s"$file did not contain a valid JSON job specification") + } + job + } + + val jobConf = CommonConf(conf) + + job + .setId(jobConf.appId) + .setName(jobConf.appName) + .addMeta("spark.nomad.role", "application") + .setJobModifyIndex(BigInteger.ZERO) + + applyDefault(job.getType)(job.setType("batch")) + applyConf(conf, NOMAD_JOB_PRIORITY, job.getPriority)(job.setPriority(_)) + applyConf(conf, REGION, job.getRegion)(job.setRegion) + + def asJavaList(commaSeparated: String): java.util.List[String] = + commaSeparated.split(",").toSeq + .map(_.trim) + .filter(_.nonEmpty) + .asJava + + conf.getOption(DATACENTERS.key) match { + case Some(explicit) => job.setDatacenters(asJavaList(explicit)) + case None if job.getDatacenters == null => DATACENTERS.defaultValue + .foreach(dcs => job.setDatacenters(asJavaList(dcs))) + } + + logInfo( + s"Will run as Nomad job [${job.getId}]" + + s" with priority ${job.getPriority}" + + s" in datacenter(s) [${job.getDatacenters.asScala.mkString(",")}]" + + Option(job.getRegion).fold("")(r => s" in region [$r]") + ) + + val driverTemplate = find(job, DriverTaskGroup) + driverCommand match { + case Some(command) => + val group = driverTemplate.getOrElse { + val g = new TaskGroup() + job.addTaskGroups(g) + g + } + val parameters = DriverTask.Parameters(command, nomadUrl) + DriverTaskGroup.configure(jobConf, conf, group, parameters) + case None => + driverTemplate.foreach(job.getTaskGroups.remove) + } + + val group = find(job, ExecutorTaskGroup).getOrElse { + val g = new TaskGroup() + job.addTaskGroups(g) + g + } + ExecutorTaskGroup.configure(jobConf, conf, group, false) + + job + } + + def utcTimestamp: String = + formatUTC(new Date(), UTC_TIMESTAMP_FORMAT) + + def applyConf[A]( + conf: SparkConf, + entry: OptionalConfigEntry[A], + getter: => AnyRef + )( + setter: A => Any + ): Unit = { + conf.get(entry).foreach(setter) + } + + def applyConf[A]( + conf: SparkConf, + entry: ConfigEntry[A], + getter: => AnyRef + )( + setter: A => Any + ): Unit = { + conf.getOption(entry.key) match { + case Some(explicit) => + setter(entry.valueConverter(explicit)) + case None => + if (getter == null) { + entry.defaultValue.foreach(setter) + } + } + } + + def find(job: Job, groupType: SparkNomadTaskGroupType): Option[TaskGroup] = { + val expectedRoles = groupType.tasks.map(_.role).toSet + val groups = Option(job.getTaskGroups).map(_.asScala).getOrElse(Nil) + .filter(g => g.getTasks != null && g.getTasks.asScala + .exists(taskMustBelongToGroup(groupType, _)) + ) + groups match { + case Seq() => None + case Seq(group) => + val tasks = group.getTasks.asScala + val tasksWithUnexpectedRoles = tasks + .filter(!taskMustBelongToGroup(groupType, _)) + .flatMap(t => SparkNomadTaskType.roleOf(t).map(r => s"task $t with role $r")) + if (tasksWithUnexpectedRoles.nonEmpty) { + throw new SparkException( + s"Tasks with roles in $expectedRoles can be mixed with role-less tasks, " + + s"but not with tasks with other roles. Task group ${group.getName} " + + s"has tasks with these roles mixed with roles $tasksWithUnexpectedRoles.") + } + groupType.tasks.foreach { taskType => + val tasksWithThisType = tasks.filter(taskType.isTypeOf) + if (tasksWithThisType.size > 1) { + val taskNames = tasksWithThisType.map(_.getName) + throw new SparkException( + s"There should be at most one template task with the ${taskType.role} role, " + + s"but task group ${group.getName} has multiple tasks with this role: $taskNames") + } + } + Some(group) + case _ => + throw new SparkException( + s"All tasks with roles in $expectedRoles should be in a single task group, " + + s"but they are split over the following groups: ${groups.map(_.getName)}" + ) + } + } + + private def applyDefault[A](getter: => AnyRef)(set: => Unit): Unit = + if (getter == null) { + set + } + + private def taskMustBelongToGroup(groupType: SparkNomadTaskGroupType, task: Task): Boolean = + groupType.tasks.exists(_.isTypeOf(task)) + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadJobController.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadJobController.scala new file mode 100644 index 0000000000000..45b8b67af43c0 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadJobController.scala @@ -0,0 +1,100 @@ +/* + * 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.nomad + +import com.hashicorp.nomad.apimodel._ +import com.hashicorp.nomad.scalasdk.NomadScalaApi + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.cluster.nomad.SparkNomadJob.CommonConf + +/** + * Manipulates a Nomad job running a Spark application. + * + * A single Nomad job is used for a Spark application running with a Nomad master. + * This includes all of the executors (and possibly shuffle servers), + * and in cluster mode also includes the Spark driver process. + */ +private[spark] class SparkNomadJobController(jobManipulator: NomadJobManipulator) extends Logging { + + def jobId: String = jobManipulator.jobId + + def startDriver(): Evaluation = { + logInfo(s"Starting driver in Nomad job ${jobId}") + jobManipulator.create() + } + + def fetchDriverLogUrls(conf: SparkConf): Option[Map[String, String]] = { + for { + allocId <- Option(conf.getenv("NOMAD_ALLOC_ID")) + taskName <- Option(conf.getenv("NOMAD_TASK_NAME")) + } yield jobManipulator.fetchLogUrlsForTask(allocId, taskName) + } + + def initialiseExecutors( + conf: SparkConf, + driverUrl: String, + count: Int + ): Unit = { + jobManipulator.updateJob(startIfNotYetRunning = count > 0) { job => + val group = SparkNomadJob.find(job, ExecutorTaskGroup).get + ExecutorTaskGroup.initialize(conf, group, driverUrl, count) + } + } + + def setExecutorCount(count: Int): Unit = { + jobManipulator.updateJob(startIfNotYetRunning = count > 0) { job => + SparkNomadJob.find(job, ExecutorTaskGroup).get + .setCount(count) + } + } + + def resolveExecutorLogUrls(reportedLogUrls: Map[String, String]): Map[String, String] = + reportedLogUrls.get(ExecutorTask.LOG_KEY_FOR_ALLOC_ID) match { + case None => + logWarning(s"Didn't find expected ${ExecutorTask.LOG_KEY_FOR_ALLOC_ID} key in " + + s"executor log URLs: $reportedLogUrls") + reportedLogUrls + case Some(allocId) => + jobManipulator.fetchLogUrlsForTask(allocId, "executor") + } + + def close(): Unit = { + jobManipulator.close() + } + +} + +private[spark] object SparkNomadJobController extends Logging { + + def initialize( + clusterModeConf: NomadClusterManagerConf, + nomad: NomadScalaApi + ): SparkNomadJobController = { + try { + val manipulator = NomadJobManipulator.fetchOrCreateJob(nomad, clusterModeConf.jobDescriptor) + new SparkNomadJobController(manipulator) + } catch { + case e: Throwable => + nomad.close() + throw e + } + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadTaskGroupType.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadTaskGroupType.scala new file mode 100644 index 0000000000000..55ee7496e452b --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadTaskGroupType.scala @@ -0,0 +1,96 @@ +/* + * 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.nomad + +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ + +import com.hashicorp.nomad.apimodel._ +import org.apache.commons.lang3.StringUtils + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.scheduler.cluster.nomad.SparkNomadJob.applyConf + +/** + * Defines configuration for a type of Nomad task group that is created by Spark. + */ +private[spark] abstract class SparkNomadTaskGroupType( + val role: String, + val tasks: SparkNomadTaskType* +) { + + val RETRY_ATTEMPTS = + ConfigBuilder(s"spark.nomad.$role.retryAttempts") + .doc(s"The number of times Nomad should retry $role task groups if they fail") + .intConf + .createWithDefault(5) + + val RETRY_DELAY_NANOS = + ConfigBuilder(s"spark.nomad.$role.retryDelay") + .doc(s"How long Nomad should wait before retrying $role task groups if they fail") + .timeConf(TimeUnit.NANOSECONDS) + .createWithDefaultString("15s") + + val RETRY_INTERVAL_NANOS = + ConfigBuilder(s"spark.nomad.$role.retryInterval") + .doc(s"Nomad's retry interval for $role task groups") + .timeConf(TimeUnit.NANOSECONDS) + .createWithDefaultString("1d") + + protected def configureCommonSettings( + jobConf: SparkNomadJob.CommonConf, + conf: SparkConf, + group: TaskGroup + ): Unit = { + + if (StringUtils.isEmpty(group.getName)) { + group.setName(role) + } + + group.setCount(1) + + val policy = Option(group.getRestartPolicy).getOrElse { + val p = new RestartPolicy + group.setRestartPolicy(p) + p + } + if (StringUtils.isEmpty(policy.getMode)) { + policy.setMode("fail") + } + applyConf(conf, RETRY_ATTEMPTS, policy.getAttempts)(policy.setAttempts(_)) + applyConf(conf, RETRY_DELAY_NANOS, policy.getDelay)(policy.setDelay(_)) + applyConf(conf, RETRY_INTERVAL_NANOS, policy.getInterval)(policy.setInterval(_)) + } + + protected def find(taskGroup: TaskGroup, taskType: SparkNomadTaskType): Option[Task] = { + taskGroup.getTasks match { + case null => None + case tasks => tasks.asScala.find(taskType.isTypeOf) + } + } + + protected def findOrAdd(group: TaskGroup, taskType: SparkNomadTaskType): Task = + find(group, taskType).getOrElse { + val task = taskType.newBlankTask() + group.addTasks(task) + task + } + +} diff --git a/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadTaskType.scala b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadTaskType.scala new file mode 100644 index 0000000000000..84e44b2d14940 --- /dev/null +++ b/resource-managers/nomad/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SparkNomadTaskType.scala @@ -0,0 +1,304 @@ +/* + * 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.nomad + +import java.net.URI +import java.util +import java.util.Collections.singletonList + +import scala.collection.JavaConverters._ + +import com.hashicorp.nomad.apimodel._ +import org.apache.commons.io.FilenameUtils +import org.apache.commons.lang3.StringUtils + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.config.{ConfigBuilder, ConfigEntry} +import org.apache.spark.network.util.ByteUnit +import org.apache.spark.scheduler.cluster.nomad.SparkNomadJob.{applyConf, JOB_TEMPLATE, SPARK_DISTRIBUTION} +import org.apache.spark.util.Utils + +/** + * Defines configuration for a type of Nomad task that is created by Spark. + */ +private[spark] abstract class SparkNomadTaskType( + val role: String, + val fullName: String, + memoryConfigEntry: ConfigEntry[Long]) { + + val CPU = + ConfigBuilder(s"spark.nomad.$role.cpu") + .doc(s"How many MHz of CPU power Nomad should reserve for $fullName tasks") + .intConf + .createWithDefault(1000) + + val MEMORY_OVERHEAD = + ConfigBuilder(s"spark.nomad.$role.memoryOverhead") + .doc("The amount of memory that should be allocated to the nomad task above the heap " + + s"memory specified by ${memoryConfigEntry.key}") + .bytesConf(ByteUnit.MiB) + .createOptional + + val NETWORK_MBITS = + ConfigBuilder(s"spark.nomad.$role.networkMBits") + .doc(s"The network bandwidth Nomad should allocate to $fullName tasks during bin packing") + .intConf + .createWithDefault(1) + + val LOG_MAX_FILES = + ConfigBuilder(s"spark.nomad.$role.logMaxFiles") + .doc(s"Number of log files Nomad should keep from $fullName tasks") + .intConf + .createWithDefault(5) + + val LOG_MAX_FILE_SIZE_MIB = + ConfigBuilder(s"spark.nomad.$role.logMaxFileSize") + .doc(s"Maximum size that Nomad should keep in log files from $fullName tasks") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("1m") + + val MEMORY_OVERHEAD_FACTOR = 0.10 + val MEMORY_OVERHEAD_MIN = 384L + + case class ConfigurablePort(label: String) { + def placeholder: String = s"$${NOMAD_PORT_$label}" + def ipPlaceholder: String = s"$${NOMAD_IP_$label}" + def placeholderInSiblingTasks(task: Task): String = s"$${NOMAD_PORT_${task.getName}_$label}" + + override def toString: String = placeholder + } + + def newBlankTask(): Task = + new Task().addMeta(SparkNomadTaskType.ROLE_META_KEY, role) + + def isTypeOf(task: Task): Boolean = + SparkNomadTaskType.roleOf(task).contains(role) + + protected def configure( + jobConf: SparkNomadJob.CommonConf, + conf: SparkConf, + task: Task, + ports: Seq[ConfigurablePort], + sparkCommand: String + ): Unit = { + + configureResources(conf, task, ports) + + if (StringUtils.isEmpty(task.getName)) { + task.setName(fullName.replaceAll(" ", "-")) + } + + def setConfigIfMissing(key: String)(value: => String) = + if (Option(task.getConfig).flatMap(config => Option(config.get(key))).isEmpty) { + task.addConfig(key, value) + } + + def configureDocker(): Unit = { + jobConf.dockerImage.foreach(setConfigIfMissing("image")(_)) + setConfigIfMissing("network_mode")("host") + setConfigIfMissing("work_dir")("/local") + jobConf.dockerAuth.foreach { properties => + task.getConfig.get("auth") match { + case null => + task.addConfig("auth", singletonList(properties.asJava)) + case authList: java.util.List[_] if authList.isEmpty => + task.addConfig("auth", singletonList(properties.asJava)) + case authList: java.util.List[_] => + val auth = authList.get(0).asInstanceOf[java.util.Map[String, AnyRef]] + properties.foreach { case (k, v) => + auth.put(k, v) + } + } + } + } + + task.getDriver match { + case "docker" => configureDocker() + case null | "" if jobConf.dockerImage.isDefined => + task.setDriver("docker") + configureDocker() + case null | "" => + task.setDriver("exec") + + if (task.getConstraints == null || + task.getConstraints.asScala.forall(_.getLTarget != "driver.java.version")) { + + task.addConstraints( + new Constraint() + .setLTarget("driver.java.version") + .setOperand(">=") + .setRTarget("1.8.0") + ) + } + case _ => + } + + if (Option(task.getEnv).flatMap(env => Option(env.get("SPARK_LOCAL_IP"))).isEmpty) { + task.addEnv("SPARK_LOCAL_IP", ports.head.ipPlaceholder) + } + + setConfigIfMissing("command") { + val sparkDistributionUrl = jobConf.sparkDistribution.getOrElse { + throw new SparkException( + s"Don't know where to find spark for $fullName task. " + + s"You must either set ${SPARK_DISTRIBUTION.key} or provide a ${JOB_TEMPLATE.key} " + + s"""with a command provided for the task with meta spark.nomad.role = "$role"""") + } + val (sparkHomeUrl, sparkArtifact) = asFileAndArtifact(jobConf, sparkDistributionUrl) + sparkArtifact.foreach(task.addArtifacts(_)) + val sparkDir = + if (sparkDistributionUrl.getScheme == "local") sparkHomeUrl.getPath + else FilenameUtils.removeExtension(sparkHomeUrl.getPath) + s"$sparkDir/bin/$sparkCommand" + } + } + + protected def appendArguments( + task: Task, + arguments: Seq[String], + removeOld: Boolean = false + ): Unit = { + Option(task.getConfig).flatMap(opts => Option(opts.get("args"))) match { + case None => task.addConfig("args", new util.ArrayList(arguments.asJava)) + case Some(javaArgs) => + val args = javaArgs.asInstanceOf[java.util.List[String]].asScala + if (removeOld) { + args.remove(args.length - arguments.length, arguments.length) + } + args.appendAll(arguments) + } + } + + protected def asFileIn(jobConf: SparkNomadJob.CommonConf, task: Task)(url: String): String = { + val (file, artifact) = asFileAndArtifact(jobConf, new URI(url)) + artifact.foreach(task.addArtifacts(_)) + file.toString + } + + protected def jvmMemory(conf: SparkConf, task: Task): String = { + val megabytes: Long = conf.getOption(memoryConfigEntry.key) match { + case Some(stringValue) => + memoryConfigEntry.valueConverter(stringValue) + case None => + val memoryWithOverhead = task.getResources.getMemoryMb + val overhead = conf.get(MEMORY_OVERHEAD).getOrElse( + math.max( + (memoryWithOverhead / (1 + 1 / MEMORY_OVERHEAD_FACTOR)).toLong, + MEMORY_OVERHEAD_MIN + ) + ) + memoryWithOverhead - overhead + } + megabytes + "m" + } + + private def configureResources( + conf: SparkConf, task: Task, ports: Seq[ConfigurablePort]): Unit = { + + val resources = Option(task.getResources).getOrElse { + val r = new Resources + task.setResources(r) + r + } + + applyConf(conf, CPU, resources.getCpu)(resources.setCpu(_)) + + (conf.getOption(memoryConfigEntry.key), conf.get(MEMORY_OVERHEAD)) match { + case (Some(explicitMemoryString), Some(explicitOverhead)) => + resources.setMemoryMb( + (memoryConfigEntry.valueConverter(explicitMemoryString) + explicitOverhead).toInt) + case _ if resources.getMemoryMb != null => + // use the value that is already set + case (_, overheadOpt) => + val memory = conf.get(memoryConfigEntry) + val overhead = overheadOpt.getOrElse( + math.max( + (MEMORY_OVERHEAD_FACTOR * memory).toLong, + MEMORY_OVERHEAD_MIN + ) + ) + resources.setMemoryMb((memory + overhead).toInt) + } + + val network = Option(resources.getNetworks).flatMap(_.asScala.headOption).getOrElse { + val n = new NetworkResource + resources.addNetworks(n) + n + } + applyConf(conf, NETWORK_MBITS, network.getMBits)(network.setMBits(_)) + + ports.foreach { port => + def alreadyContainsPort(ports: java.util.List[Port]): Boolean = + Option(ports).exists(_.asScala.exists(_.getLabel == port.label)) + + if (!alreadyContainsPort(network.getDynamicPorts) + && !alreadyContainsPort(network.getReservedPorts) + ) { + network.addDynamicPorts(new Port().setLabel(port.label)) + } + } + + val logConfig = Option(task.getLogConfig).getOrElse { + val l = new LogConfig + task.setLogConfig(l) + l + } + applyConf(conf, LOG_MAX_FILES, logConfig.getMaxFiles)(logConfig.setMaxFiles(_)) + applyConf(conf, LOG_MAX_FILE_SIZE_MIB, logConfig.getMaxFileSizeMb)(s => + logConfig.setMaxFileSizeMb(s.toInt) + ) + } + + private def asFileAndArtifact( + jobConf: SparkNomadJob.CommonConf, + url: URI + ): (URI, Option[TaskArtifact]) = { + url.getScheme match { + case "local" => url -> None + case null | "file" => + throw new SparkException( + "Bare paths and the \"file:\" URL scheme are not supported in Nomad cluster mode. " + + "Use a remote (e.g. HTTP/S) URL that nomad can download, or use a \"local:\" URL " + + "if the resource is available at a well-known path on all nomad clients that might " + + "run the task. Offending path/URL: " + url + ) + case "http" | "https" | "s3" => + val workDir = + if (jobConf.dockerImage.isDefined) "/local/" + else "/" + val file = new URI("file://" + workDir + Utils.decodeFileNameInURI(url)) + val artifact = new TaskArtifact() + .setRelativeDest(workDir) + .setGetterSource(url.toString) + file -> Some(artifact) + } + } + +} + +private[spark] object SparkNomadTaskType { + + val ROLE_META_KEY = "spark.nomad.role" + + def roleOf(task: Task): Option[String] = + task.getMeta match { + case null => None + case meta => Option(meta.get(SparkNomadTaskType.ROLE_META_KEY)) + } + +} diff --git a/resource-managers/nomad/src/test/resources/log4j.properties b/resource-managers/nomad/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..d13454d5ae5d5 --- /dev/null +++ b/resource-managers/nomad/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=DEBUG, 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/nomad/src/test/resources/org/apache/spark/scheduler/cluster/nomad/job-template.json b/resource-managers/nomad/src/test/resources/org/apache/spark/scheduler/cluster/nomad/job-template.json new file mode 100644 index 0000000000000..93cfe6b47cc1d --- /dev/null +++ b/resource-managers/nomad/src/test/resources/org/apache/spark/scheduler/cluster/nomad/job-template.json @@ -0,0 +1,182 @@ +{ + "Job": { + "Region": null, + "ID": "template", + "ParentID": null, + "Name": "template", + "Type": null, + "Priority": null, + "AllAtOnce": null, + "Datacenters": null, + "Constraints": null, + "TaskGroups": [ + { + "Name": "driver", + "Count": null, + "Constraints": null, + "Tasks": [ + { + "Name": "drive", + "Driver": "", + "User": "", + "Config": null, + "Constraints": null, + "Env": null, + "Services": null, + "Resources": null, + "Meta": { + "spark.nomad.role": "driver" + }, + "KillTimeout": null, + "LogConfig": null, + "Artifacts": null, + "Vault": null, + "Templates": null, + "DispatchPayload": null, + "Leader": false + }, + { + "Name": "some-driver-sidecar", + "Driver": "exec", + "User": "", + "Config": { + "args": [ + "-c", + "echo hi" + ], + "command": "/bin/bash" + }, + "Constraints": null, + "Env": null, + "Services": null, + "Resources": { + "CPU": 20, + "MemoryMB": 10, + "DiskMB": null, + "IOPS": null, + "Networks": [ + { + "Public": false, + "CIDR": "", + "ReservedPorts": null, + "DynamicPorts": [ + { + "Label": "foo", + "Value": 0 + } + ], + "IP": "", + "MBits": 1 + } + ] + }, + "Meta": null, + "KillTimeout": null, + "LogConfig": { + "MaxFiles": 1, + "MaxFileSizeMB": 1 + }, + "Artifacts": null, + "Vault": null, + "Templates": null, + "DispatchPayload": null, + "Leader": false + } + ], + "RestartPolicy": null, + "EphemeralDisk": null, + "Meta": null + }, + { + "Name": "executor", + "Count": null, + "Constraints": null, + "Tasks": [ + { + "Name": "executor", + "Driver": "", + "User": "", + "Config": null, + "Constraints": null, + "Env": null, + "Services": null, + "Resources": null, + "Meta": { + "spark.nomad.role": "executor" + }, + "KillTimeout": null, + "LogConfig": null, + "Artifacts": null, + "Vault": null, + "Templates": null, + "DispatchPayload": null, + "Leader": false + }, + { + "Name": "some-executor-sidecar", + "Driver": "exec", + "User": "", + "Config": { + "args": [ + "-c", + "echo hi" + ], + "command": "/bin/bash" + }, + "Constraints": null, + "Env": null, + "Services": null, + "Resources": { + "CPU": 20, + "MemoryMB": 10, + "DiskMB": null, + "IOPS": null, + "Networks": [ + { + "Public": false, + "CIDR": "", + "ReservedPorts": null, + "DynamicPorts": [ + { + "Label": "bar", + "Value": 0 + } + ], + "IP": "", + "MBits": 1 + } + ] + }, + "Meta": null, + "KillTimeout": null, + "LogConfig": { + "MaxFiles": 1, + "MaxFileSizeMB": 1 + }, + "Artifacts": null, + "Vault": null, + "Templates": null, + "DispatchPayload": null, + "Leader": false + } + ], + "RestartPolicy": null, + "EphemeralDisk": null, + "Meta": null + } + ], + "Update": null, + "Periodic": null, + "ParameterizedJob": null, + "Payload": null, + "Meta": { + "hi": "there" + }, + "VaultToken": null, + "Status": null, + "StatusDescription": null, + "CreateIndex": null, + "ModifyIndex": null, + "JobModifyIndex": null + } +} diff --git a/resource-managers/nomad/src/test/resources/org/apache/spark/scheduler/cluster/nomad/job-template.nomad b/resource-managers/nomad/src/test/resources/org/apache/spark/scheduler/cluster/nomad/job-template.nomad new file mode 100644 index 0000000000000..5f07814ceb691 --- /dev/null +++ b/resource-managers/nomad/src/test/resources/org/apache/spark/scheduler/cluster/nomad/job-template.nomad @@ -0,0 +1,84 @@ +# +# 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. +# + +job "template" { + + meta { + "hi" = "there" + } + + group "driver" { + + task "drive" { + meta { + "spark.nomad.role" = "driver" + } + } + + task "some-driver-sidecar" { + driver = "exec" + config { + command = "/bin/bash" + args = [ "-c", "echo hi" ] + } + resources { + cpu = 20 + memory = 10 + network { + mbits = 1 + port "foo" {} + } + } + logs { + max_files = 1 + max_file_size = 1 + } + } + + } + + group "executor" { + + task "executor" { + meta { + "spark.nomad.role" = "executor" + } + } + + task "some-executor-sidecar" { + driver = "exec" + config { + command = "/bin/bash" + args = [ "-c", "echo hi" ] + } + resources { + cpu = 20 + memory = 10 + network { + mbits = 1 + port "bar" {} + } + } + logs { + max_files = 1 + max_file_size = 1 + } + } + + } + +} diff --git a/resource-managers/nomad/src/test/scala/org/apache/spark/deploy/nomad/BaseNomadClusterSuite.scala b/resource-managers/nomad/src/test/scala/org/apache/spark/deploy/nomad/BaseNomadClusterSuite.scala new file mode 100644 index 0000000000000..b21ecc961b85d --- /dev/null +++ b/resource-managers/nomad/src/test/scala/org/apache/spark/deploy/nomad/BaseNomadClusterSuite.scala @@ -0,0 +1,318 @@ +/* + * 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.nomad + +import java.io._ +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.Properties + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.reflect.ClassTag + +import com.google.common.io.Files +import org.apache.commons.lang3.SerializationUtils +import org.apache.http.HttpHost +import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark._ +import org.apache.spark.launcher._ +import org.apache.spark.launcher.SparkAppHandle.State +import org.apache.spark.launcher.SparkAppHandle.State.FINISHED +import org.apache.spark.scheduler.cluster.nomad.{SparkNomadJob, TestApplication} +import org.apache.spark.util.Utils + +abstract class BaseNomadClusterSuite extends SparkFunSuite with BeforeAndAfterEach with Matchers { + + private val moduleDir = { + val relativeToProjectRoot = new File(new File("resource-managers"), "nomad") + if (relativeToProjectRoot.exists()) relativeToProjectRoot + else new File("") + }.getAbsoluteFile + + private val projectDir = moduleDir.getParentFile.getParentFile + + /** + * The spark.nomad.test.host system property can is useful when the test machine is separated from + * the Nomad cluster by NAT. + */ + val hostnameAsSeenFromNomad: Option[String] = sys.props.get("spark.nomad.test.host") + + /** + * An external Nomad server HTTP URL to test against. + * When not specified, NOMAD_ADDR or the default (http://localhost/4646) will be used. + */ + val nomadTestAddress: Option[HttpHost] = + sys.props.get("spark.nomad.test.url") + .map(HttpHost.create) + + /** + * The docker image to use to run Spark in Nomad tasks. + * When not specified, the `exec` driver is used instead. + */ + val dockerImage: Option[String] = sys.props.get("spark.nomad.test.dockerImage") + + /** + * The Spark distribution to use in Nomad tasks. + * When not specified, the test looks for a distribution in the working directory + * and serves it over HTTP. + */ + val sparkDistribution: String = sys.props.get("spark.nomad.test.sparkDistribution").getOrElse { + uniqueFileMatchingPattern(projectDir, "spark-.+?-bin-.+?\\.tgz") + .getOrElse(sys.error("spark.nomad.test.sparkDistribution not set and can't find file in " + + projectDir + " matching spark-.+?-bin-.+?\\.tgz")) + .toString + } + + private var oldSystemProperties: Properties = _ + protected var tempDir: File = _ + private var logConfDir: File = _ + + override def beforeAll() { + super.beforeAll() + + oldSystemProperties = SerializationUtils.clone(System.getProperties) + + tempDir = Utils.createTempDir() + + // log4j configuration for the processes spawned by the tests, + // so that their output doesn't overwrite unit-tests.log. + logConfDir = new File(tempDir, "log4j") + logConfDir.mkdir() + Files.write( + """log4j.rootCategory=DEBUG, console + |log4j.appender.console=org.apache.log4j.ConsoleAppender + |log4j.appender.console.target=System.err + |log4j.appender.console.layout=org.apache.log4j.PatternLayout + |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + |log4j.logger.org.apache.hadoop=WARN + |log4j.logger.org.apache.http=WARN + |log4j.logger.org.apache.http.wire=ERROR + |log4j.logger.org.eclipse.jetty=WARN + |log4j.logger.org.mortbay=WARN + |log4j.logger.org.spark_project.jetty=WARN + |""".stripMargin, + new File(logConfDir, "log4j.properties"), + UTF_8 + ) + } + + override def afterAll() { + System.setProperties(oldSystemProperties) + super.afterAll() + } + + + var httpServer: HttpTestServer = _ + + override protected def beforeEach(): Unit = { + httpServer = HttpTestServer(hostnameAsSeenFromNomad) + } + + override protected def afterEach(): Unit = { + httpServer.close() + } + + + sealed abstract class DeployMode(override val toString: String) + + case object ClientMode extends DeployMode("client") + + case object ClusterMode extends DeployMode("cluster") + + sealed trait MainArtifact + + case class JarWithMainClass(jar: File, mainClass: String) extends MainArtifact + + case class FileResource(file: File) extends MainArtifact + + def mainClassName[A](mainClass: Class[A]): String = { + assert(mainClass.getDeclaredMethod("main", classOf[Array[String]]) != null) + mainClass.getName.stripSuffix("$") + } + + lazy val testAppJar = { + val jarNamePattern = "spark-nomad-test-apps_.*(? file.getAbsolutePath + case ClusterMode => httpServer.addMapping("/some/path/" + file.getName, file) + } + } + + def runSpark( + deployMode: DeployMode, + mainArtifact: MainArtifact, + appArgs: Seq[String] = Nil, + sparkArgs: Seq[(String, String)] = Nil, + extraClassPath: Seq[String] = Nil, + extraJars: Seq[File] = Nil, + extraFiles: Seq[File] = Nil, + extraConf: Map[String, String] = Map(), + extraEnv: Map[String, String] = Map()): State = { + + val propsFile = createConfFile(extraClassPath = extraClassPath, extraConf = extraConf) + + val launcher = new SparkLauncher(extraEnv.asJava) + mainArtifact match { + case JarWithMainClass(jar, klass) => + launcher.setMainClass(klass) + launcher.setAppResource(resource(deployMode, jar)) + case FileResource(file) => + launcher.setAppResource(resource(deployMode, file)) + } + launcher.setSparkHome(sys.props("spark.test.home")) + .setMaster(nomadTestAddress.fold("nomad")("nomad:" +)) + .setDeployMode(deployMode.toString) + .setConf("spark.executor.instances", "1") + .setPropertiesFile(propsFile) + .addSparkArg("--verbose") + .addSparkArg("--driver-memory", "600m") + .addSparkArg("--executor-memory", "500m") + .addAppArgs(appArgs.toArray: _*) + + if (deployMode == ClientMode) hostnameAsSeenFromNomad.foreach { hostname => + launcher.setConf("spark.driver.host", hostname) + launcher.setConf("spark.driver.bindAddress", "0.0.0.0") + } + + sparkArgs.foreach { case (name, value) => + if (value != null) { + launcher.addSparkArg(name, value) + } else { + launcher.addSparkArg(name) + } + } + extraJars.foreach(jar => launcher.addJar(resource(deployMode, jar))) + extraFiles.foreach(file => launcher.addFile(resource(deployMode, file))) + + launcher.setVerbose(true) + + val handle = launcher.startApplication() + try { + eventually(timeout(3 minutes), interval(1 second)) { + val state = handle.getState() + logDebug("CURRENT STATE IS " + state) + assert(state.isFinal()) + } + } finally { + handle.kill() + } + + handle.getState() + } + + def checkResult( + finalState: State, + expectedPathValues: (String, String)*): Unit = { + val expectedPathValueOptions = + expectedPathValues.map { + case (path, expectedValue) => (path, Some(expectedValue)) + } + val actualServerContents = expectedPathValues.map { + case (path, expectedValue) => (path, httpServer.valuePutToPath(path)) + } + (finalState, actualServerContents) should be(FINISHED, expectedPathValueOptions) + } + + def createConfFile( + extraClassPath: Seq[String] = Nil, + extraConf: Map[String, String] = Map()): String = { + val props = new Properties() + + dockerImage.foreach(props.put(SparkNomadJob.DOCKER_IMAGE.key, _)) + + props.put(SparkNomadJob.SPARK_DISTRIBUTION.key, + if (sparkDistribution.indexOf(":") == -1) { + val file = new File(sparkDistribution) + httpServer.addMapping("/" + file.getName, file) + } else { + sparkDistribution + } + ) + + val testClasspath = + new TestClasspathBuilder().buildClassPath( + logConfDir.getAbsolutePath + + File.pathSeparator + + extraClassPath.mkString(File.pathSeparator) + ) + .asScala + .mkString(File.pathSeparator) + + props.put("spark.driver.extraClassPath", testClasspath) + props.put("spark.executor.extraClassPath", testClasspath) + + // SPARK-4267: make sure java options are propagated correctly. + props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") + props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"") + + sys.props.foreach { case (k, v) => + if (k.startsWith("spark.")) { + props.setProperty(k, v) + } + } + extraConf.foreach { case (k, v) => props.setProperty(k, v) } + + val propsFile = File.createTempFile("spark", ".properties", tempDir) + val writer = new OutputStreamWriter(new FileOutputStream(propsFile), UTF_8) + try props.store(writer, "Spark properties.") + finally writer.close() + propsFile.getAbsolutePath + } + + private def uniqueFileMatchingPattern(directory: File, pattern: String): Option[File] = { + directory.listFiles( + new FilenameFilter { + override def accept(dir: File, name: String): Boolean = + name.matches(pattern) + } + ) match { + case null => sys.error(s"Can't find directory $directory") + case Array() => None + case Array(file) => Some(file) + case files => sys.error( + files.mkString( + s"Expected 1 file in $directory matching $pattern, found ${files.length}: ", ", ", "") + ) + } + } + +} diff --git a/resource-managers/nomad/src/test/scala/org/apache/spark/deploy/nomad/HttpTestServer.scala b/resource-managers/nomad/src/test/scala/org/apache/spark/deploy/nomad/HttpTestServer.scala new file mode 100644 index 0000000000000..68dc88a8b6ec5 --- /dev/null +++ b/resource-managers/nomad/src/test/scala/org/apache/spark/deploy/nomad/HttpTestServer.scala @@ -0,0 +1,104 @@ +/* + * 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.nomad + +import java.io.File +import java.nio.charset.StandardCharsets +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} + +import org.apache.commons.codec.digest.DigestUtils +import org.apache.commons.io.{FileUtils, IOUtils} +import org.eclipse.jetty.server.{AbstractNetworkConnector, Request, Server} +import org.eclipse.jetty.server.handler.AbstractHandler + +import org.apache.spark.util.Utils + +class HttpTestServer(hostnameInUrls: String) { + import HttpTestServer._ + + private var fileAndChecksumMappings = Map.empty[String, (File, String)] + private var putMappings = Map.empty[String, Array[Byte]] + + private val server = new Server(0) + server.setHandler(new AbstractHandler { + override def handle(target: String, baseRequest: Request, request: HttpServletRequest, + response: HttpServletResponse): Unit = { + + baseRequest.getMethod match { + case "GET" => + putMappings.get(baseRequest.getRequestURI) match { + case Some(bytes) => + response.setStatus(200) + IOUtils.write(bytes, response.getOutputStream) + baseRequest.setHandled(true) + case None => + fileAndChecksumMappings.get(baseRequest.getRequestURI) + .foreach { case (file, _) => + response.setStatus(200) + FileUtils.copyFile(file, response.getOutputStream) + baseRequest.setHandled(true) + } + } + case "PUT" => + response.setStatus(201) + putMappings += baseRequest.getRequestURI -> IOUtils.toByteArray(request.getInputStream) + baseRequest.setHandled(true) + case method => + response.setStatus(405) + baseRequest.setHandled(true) + } + } + }) + server.start() + + def port: Int = + server.getConnectors.head.asInstanceOf[AbstractNetworkConnector].getLocalPort + + def addMapping(path: String, file: File): String = { + require(file.isFile, s"$file must be a file, but isn't") + fileAndChecksumMappings += path -> (file, checksum(file)) + url(path) + } + + def url(path: String): String = { + val query = fileAndChecksumMappings.get(path) match { + case None => "" + case Some((_, checksum)) => s"?checksum=$checksum" + } + s"http://$hostnameInUrls:$port$path$query" + } + + def valuePutToPath(path: String): Option[String] = + putMappings.get(path) + .map(new String(_, StandardCharsets.UTF_8)) + + def close(): Unit = + server.stop() + +} + +object HttpTestServer { + + def apply(hostnameInUrls: Option[String]): HttpTestServer = + new HttpTestServer(hostnameInUrls.getOrElse(Utils.localHostNameForURI())) + + private def checksum(file: File): String = { + "md5:" + Utils.tryWithResource(FileUtils.openInputStream(file))(DigestUtils.md5Hex) + } + +} diff --git a/resource-managers/nomad/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala b/resource-managers/nomad/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala new file mode 100644 index 0000000000000..c66fb4b758bce --- /dev/null +++ b/resource-managers/nomad/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala @@ -0,0 +1,36 @@ +/* + * 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.launcher + +import java.util.{ List => JList, Map => JMap } + +/** + * Exposes AbstractCommandBuilder to the Nomad tests, so that they can build classpaths the same + * way other cluster managers do. + */ +private[spark] class TestClasspathBuilder extends AbstractCommandBuilder { + + childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, sys.props("spark.test.home")) + + override def buildClassPath(extraCp: String): JList[String] = + super.buildClassPath(extraCp) + + override def buildCommand(env: JMap[String, String]): JList[String] = + throw new UnsupportedOperationException() + +} diff --git a/resource-managers/nomad/src/test/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterSuite.scala b/resource-managers/nomad/src/test/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterSuite.scala new file mode 100644 index 0000000000000..a122b974ef16e --- /dev/null +++ b/resource-managers/nomad/src/test/scala/org/apache/spark/scheduler/cluster/nomad/NomadClusterSuite.scala @@ -0,0 +1,389 @@ +/* + * 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.nomad + +import java.io.File +import java.nio.charset.StandardCharsets.UTF_8 + +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.apache.commons.io.IOUtils +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark._ +import org.apache.spark.deploy.nomad.BaseNomadClusterSuite +import org.apache.spark.launcher._ +import org.apache.spark.tags.ExtendedNomadTest + +/** + * Integration tests for Nomad. + * + * These tests will run on the nomad cluster whose URL (e.g. http://127.0.0.1:4646) is given in the + * nomad.test.url system property. + * + * These tests require the spark-distribution tgz to have been built and to be available over HTTP/S + * at the location provided in the nomad.test.url system property. + */ +@ExtendedNomadTest +class NomadClusterSuite extends BaseNomadClusterSuite { + + test("run in client mode") { + val finalState = runSpark( + ClientMode, + nomadTestApp[ConfigurationCheckingSparkApp.type], + appArgs = Seq(httpServer.url("/result")) + ) + checkResult(finalState, "/result" -> "SUCCESS without driver logs with 1 executor(s)") + } + + test("run in cluster mode") { + val finalState = runSpark( + ClusterMode, + nomadTestApp[ConfigurationCheckingSparkApp.type], + appArgs = Seq(httpServer.url("/result")) + ) + checkResult(finalState, "/result" -> "SUCCESS with driver logs with 1 executor(s)") + } + + test("run in client mode with additional jar") { + testWithAdditionalJar(ClientMode) + } + + test("run in cluster mode with additional jar") { + testWithAdditionalJar(ClusterMode) + } + + private def testWithAdditionalJar(deployMode: DeployMode): Unit = { + val additionalJar = new File(TestUtils.createJarWithFiles(Map( + "test.resource" -> "RESOURCE CONTENTS" + ), tempDir).getFile) + + val finalState = runSpark( + deployMode, + nomadTestApp[ResourceUploaderTestApp.type], + appArgs = Seq(httpServer.url("/driver"), httpServer.url("/executor")), + extraJars = Seq(additionalJar) + ) + checkResult(finalState, + "/driver" -> "RESOURCE CONTENTS", + "/executor" -> "RESOURCE CONTENTS" + ) + } + + + test("run in client mode with additional file") { + testWithAdditionalFile(ClientMode) + } + + test("run in cluster mode with additional file") { + testWithAdditionalFile(ClusterMode) + } + + private def testWithAdditionalFile(deployMode: DeployMode): Unit = { + val additionalFile = createFile("additionalFile", ".txt", "FILE CONTENTS") + + val finalState = runSpark( + deployMode, + nomadTestApp[FileDistributionTestDriver.type], + appArgs = Seq( + deployMode match { + case ClientMode => additionalFile.getAbsolutePath + case ClusterMode => additionalFile.getName + }, + httpServer.url("/driver"), + additionalFile.getName, + httpServer.url("/executor")), + extraFiles = Seq(additionalFile) + ) + checkResult(finalState, + "/driver" -> "FILE CONTENTS", + "/executor" -> "FILE CONTENTS" + ) + } + + + test("run in cluster mode unsuccessfully") { + // Don't provide arguments so the driver will fail. + val finalState = runSpark( + ClusterMode, + nomadTestApp[ConfigurationCheckingSparkApp.type] + ) + finalState should be (SparkAppHandle.State.FAILED) + } + + test("failure in cluster after sc initialization is reported to launcher") { + val finalState = runSpark(ClusterMode, nomadTestApp[ThrowExceptionAfterContextInit.type], + extraConf = Map("spark.nomad.driver.retryAttempts" -> "2") + ) + finalState should be (SparkAppHandle.State.FAILED) + } + + + test("user class path first in client mode") { + testUserClassPathFirst(ClientMode) + } + + test("user class path first in cluster mode") { + testUserClassPathFirst(ClusterMode) + } + + private def testUserClassPathFirst(deployMode: DeployMode): Unit = { + + val extraClassPathJar = TestUtils.createJarWithFiles(Map( + "test.resource" -> "EXTRA CLASSPATH CONTENTS" + ), tempDir) + + val userJar = new File(TestUtils.createJarWithFiles(Map( + "test.resource" -> "USER JAR CONTENTS" + ), tempDir).getFile) + + val finalState = runSpark( + deployMode, + nomadTestApp[ResourceUploaderTestApp.type], + appArgs = Seq(httpServer.url("/driver"), httpServer.url("/executor")), + extraClassPath = Seq(extraClassPathJar.getPath), // TODO: this test is wrong for cluster mode + extraJars = Seq(userJar), + extraConf = Map( + "spark.driver.userClassPathFirst" -> "true", + "spark.executor.userClassPathFirst" -> "true" + ) + ) + checkResult(finalState, + "/driver" -> "USER JAR CONTENTS", + "/executor" -> "USER JAR CONTENTS" + ) + } + + + test("monitor app using launcher library") { + val propsFile = createConfFile() + val mainClass = mainClassName(NomadLauncherTestApp.getClass) + val handle = new SparkLauncher() + .setSparkHome(sys.props("spark.test.home")) + .setConf("spark.ui.enabled", "false") + .setPropertiesFile(propsFile) + .setMaster(nomadTestAddress.fold("nomad")("nomad:" +)) + .setDeployMode("client") + .setAppResource(SparkLauncher.NO_RESOURCE) + .setMainClass(mainClass) + .startApplication() + + try { + eventually(timeout(30 seconds), interval(100 millis)) { + handle.getState() should be (SparkAppHandle.State.RUNNING) + } + + handle.getAppId() should not be (null) + handle.getAppId() should startWith (mainClass + "-") + handle.stop() + + eventually(timeout(30 seconds), interval(100 millis)) { + handle.getState() should be (SparkAppHandle.State.KILLED) + } + } finally { + handle.kill() + } + } + + test("use spark auth in client mode") { + val finalState = runSpark( + ClientMode, + nomadTestApp[ConfigurationCheckingSparkApp.type], + appArgs = Seq(httpServer.url("/result")), + extraConf = Map("spark.authenticate" -> "true") + ) + checkResult(finalState, "/result" -> "SUCCESS without driver logs with 1 executor(s)") + } + + test("use spark auth in cluster mode") { + val finalState = runSpark( + ClusterMode, + nomadTestApp[ConfigurationCheckingSparkApp.type], + appArgs = Seq(httpServer.url("/result")), + extraConf = Map("spark.authenticate" -> "true") + ) + checkResult(finalState, "/result" -> "SUCCESS with driver logs with 1 executor(s)") + } + + test("streaming app in client mode") { + val finalState = runSpark( + ClientMode, + nomadTestApp[StreamingApp.type], + appArgs = Seq(httpServer.url("/result")) + ) + checkResult(finalState, "/result" -> "success") + } + + test("streaming app in cluster mode") { + val finalState = runSpark( + ClusterMode, + nomadTestApp[StreamingApp.type], + appArgs = Seq(httpServer.url("/result")) + ) + checkResult(finalState, "/result" -> "success") + } + + test("writing to disk in client mode") { + val finalState = runSpark( + ClientMode, + nomadTestApp[LocalDiskWritingApp.type], + appArgs = Seq("NO DRIVER WRITE", httpServer.url("/executor")) + ) + checkResult(finalState, "/executor" -> "EXEC") + } + + test("writing to disk in cluster mode") { + val finalState = runSpark( + ClusterMode, + nomadTestApp[LocalDiskWritingApp.type], + appArgs = Seq(httpServer.url("/driver"), httpServer.url("/executor")) + ) + checkResult(finalState, "/driver" -> "DRIVE", "/executor" -> "EXEC") + } + + test("persisting RDD to disk in client mode") { + val finalState = runSpark( + ClientMode, + nomadTestApp[DiskPersistedRddTestDriver.type], + appArgs = Seq(httpServer.url("/result")) + ) + checkResult(finalState, "/result" -> ((123456 + 234567) * 2).toString) + } + + test("persisting RDD to disk in cluster mode") { + val finalState = runSpark( + ClusterMode, + nomadTestApp[DiskPersistedRddTestDriver.type], + appArgs = Seq(httpServer.url("/result")) + ) + checkResult(finalState, "/result" -> ((123456 + 234567) * 2).toString) + } + + test("writing file to disk in client mode") { + val finalState = runSpark( + ClientMode, + nomadTestApp[SaveToDiskTestApp.type], + appArgs = Seq(httpServer.url("/driver"), httpServer.url("/executor")) + ) + checkResult(finalState, + "/driver" -> "CONTENTS WRITTEN TO DRIVER FILE", + "/executor" -> "CONTENTS WRITTEN TO EXECUTOR FILE") + } + + test("writing file to disk in cluster mode") { + val finalState = runSpark( + ClusterMode, + nomadTestApp[SaveToDiskTestApp.type], + appArgs = Seq(httpServer.url("/driver"), httpServer.url("/executor")) + ) + checkResult(finalState, + "/driver" -> "CONTENTS WRITTEN TO DRIVER FILE", + "/executor" -> "CONTENTS WRITTEN TO EXECUTOR FILE") + } + + test("dynamic allocation of executors in client mode") { + val finalState = runSpark( + ClientMode, + nomadTestApp[DiskPersistedRddTestDriver.type], + appArgs = Seq(httpServer.url("/result")), + extraConf = Map( + "spark.dynamicAllocation.enabled" -> "true", + "spark.shuffle.service.enabled" -> "true" + ) + ) + checkResult(finalState, "/result" -> ((123456 + 234567) * 2).toString) + } + + test("dynamic allocation of executors in cluster mode") { + val finalState = runSpark( + ClusterMode, + nomadTestApp[DiskPersistedRddTestDriver.type], + appArgs = Seq(httpServer.url("/result")), + extraConf = Map( + "spark.dynamicAllocation.enabled" -> "true", + "spark.shuffle.service.enabled" -> "true" + ) + ) + checkResult(finalState, "/result" -> ((123456 + 234567) * 2).toString) + } + + test("use a job template in client mode") { + val jobTemplate = IOUtils.toString(getClass.getResourceAsStream("job-template.json"), UTF_8) + val finalState = runSpark( + ClientMode, + nomadTestApp[EnvironmentPostingTestDriver.type], + appArgs = Seq(httpServer.url("/driver/"), httpServer.url("/executor/"), + "NOMAD_ADDR_some_driver_sidecar_foo", + "NOMAD_ADDR_some_executor_sidecar_bar" + ), + extraConf = Map( + "spark.nomad.job.template" -> createFile("job-template", ".json", jobTemplate).toString + ) + ) + checkResult(finalState, + "/driver/NOMAD_ADDR_some_driver_sidecar_foo" -> "NOT SET", + "/driver/NOMAD_ADDR_some_executor_sidecar_bar" -> "NOT SET", + "/executor/NOMAD_ADDR_some_driver_sidecar_foo" -> "NOT SET") + + val executorBar = httpServer.valuePutToPath("/executor/NOMAD_ADDR_some_executor_sidecar_bar") + executorBar.get should not be "NOT SET" + } + + test("use a job template in cluster mode") { + val jobTemplate = IOUtils.toString(getClass.getResourceAsStream("job-template.json"), UTF_8) + val finalState = runSpark( + ClusterMode, + nomadTestApp[EnvironmentPostingTestDriver.type], + appArgs = Seq(httpServer.url("/driver/"), httpServer.url("/executor/"), + "NOMAD_ADDR_some_driver_sidecar_foo", + "NOMAD_ADDR_some_executor_sidecar_bar" + ), + extraConf = Map( + "spark.nomad.job.template" -> createFile("job-template", ".json", jobTemplate).toString + ) + ) + checkResult(finalState, + "/driver/NOMAD_ADDR_some_executor_sidecar_bar" -> "NOT SET", + "/executor/NOMAD_ADDR_some_driver_sidecar_foo" -> "NOT SET") + + val driverFoo = httpServer.valuePutToPath("/driver/NOMAD_ADDR_some_driver_sidecar_foo") + val executorBar = httpServer.valuePutToPath("/executor/NOMAD_ADDR_some_executor_sidecar_bar") + + driverFoo.get should not be "NOT SET" + executorBar.get should not be "NOT SET" + } + + +} + + +private object NomadLauncherTestApp { + + def main(args: Array[String]): Unit = { + // Do not stop the application; the test will stop it using the launcher lib. Just run a task + // that will prevent the process from exiting. + val sc = new SparkContext(new SparkConf()) + sc.parallelize(Seq(1)).foreach { i => + this.synchronized { + wait() + } + } + } + +} diff --git a/resource-managers/nomad/src/test/scala/org/apache/spark/scheduler/cluster/nomad/NomadPythonSuite.scala b/resource-managers/nomad/src/test/scala/org/apache/spark/scheduler/cluster/nomad/NomadPythonSuite.scala new file mode 100644 index 0000000000000..44ebdee2028ea --- /dev/null +++ b/resource-managers/nomad/src/test/scala/org/apache/spark/scheduler/cluster/nomad/NomadPythonSuite.scala @@ -0,0 +1,102 @@ +/* + * 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.nomad + +import java.io.File +import java.nio.charset.StandardCharsets + +import com.google.common.io.Files + +import org.apache.spark.TestUtils +import org.apache.spark.deploy.nomad.BaseNomadClusterSuite +import org.apache.spark.tags.ExtendedNomadTest + +@ExtendedNomadTest +class NomadPythonSuite extends BaseNomadClusterSuite { + + test("run Python application in client mode") { + testPySpark(ClientMode) + } + + test("run Python application in cluster mode") { + testPySpark(ClusterMode) + } + + private def testPySpark(deployMode: DeployMode): Unit = { + val moduleDir = deployMode match { + case ClientMode => + tempDir + case ClusterMode => + val subdir = new File(tempDir, "pyModules") + subdir.mkdir() + subdir + } + val pyModule = new File(moduleDir, "mod1.py") + Files.write(TEST_PYMODULE, pyModule, StandardCharsets.UTF_8) + + val mod2Archive = + new File(TestUtils.createJarWithFiles(Map("mod2.py" -> TEST_PYMODULE), moduleDir).toURI) + + val finalState = runSpark( + deployMode, + FileResource(createFile("test", ".py", TEST_PYFILE)), + sparkArgs = Seq( + "--py-files" -> Seq(pyModule, mod2Archive).map(resource(deployMode, _)).mkString(",") + ), + appArgs = Seq(httpServer.url("/result")) + ) + checkResult(finalState, "/result" -> "success") + } + + private val TEST_PYFILE = + """import httplib + |from urlparse import urlparse + |def put(url_string, content): + | url = urlparse(url_string) + | if (url.scheme != "http"): + | raise Exception("Url scheme needs to be http.") + | conn = httplib.HTTPConnection(url.hostname, url.port) + | conn.request("PUT", url.path, content) + | response = conn.getresponse() + | print "PUT", url, "->", response.status, response.reason + | + |import sys + |import mod1 + |import mod2 + |from operator import add + |from pyspark import SparkConf, SparkContext + |if __name__ == "__main__": + | if len(sys.argv) != 2: + | print >> sys.stderr, "Usage: test.py [result URL]" + | exit(-1) + | sc = SparkContext(conf=SparkConf()) + | result = "failure" + | rdd = sc.parallelize(range(10)).map(lambda x: x * mod1.func() * mod2.func()) + | cnt = rdd.count() + | if cnt == 10: + | result = "success" + | put(sys.argv[1], result) + | sc.stop() + |""".stripMargin + + private val TEST_PYMODULE = + """def func(): + | return 42 + |""".stripMargin + +} diff --git a/resource-managers/nomad/test-apps/pom.xml b/resource-managers/nomad/test-apps/pom.xml new file mode 100644 index 0000000000000..f08f8002d21b0 --- /dev/null +++ b/resource-managers/nomad/test-apps/pom.xml @@ -0,0 +1,57 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../../pom.xml + + + spark-nomad-test-apps_2.11 + jar + Spark Project Nomad Test Apps + + nomad-test-apps + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.httpcomponents + httpclient + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ConfigurationCheckingSparkApp.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ConfigurationCheckingSparkApp.scala new file mode 100644 index 0000000000000..6da9ab7f39a77 --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ConfigurationCheckingSparkApp.scala @@ -0,0 +1,78 @@ +/* + * 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.nomad + +import java.net.URL + +import org.apache.spark.{SparkConf, SparkContext} + +private[spark] object ConfigurationCheckingSparkApp extends TestApplication { + + val WAIT_TIMEOUT_MILLIS = 10000 + + def main(args: Array[String]): Unit = { + checkArgs(args)("result_url") + val Array(resultUrl) = args + + val sc = new SparkContext( + new SparkConf() + .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) + .setAppName("Nomad \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns") + ) + try { + httpPut(resultUrl) { + val conf = sc.getConf + + val data = sc.parallelize(1 to 4, 4).collect().toSet + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + assertEquals(data, Set(1, 2, 3, 4)) + + // verify log URLs are present + val Seq(listener) = sc.listenerBus.findListenersByClass[SaveExecutorInfo]() + val executorInfos = listener.addedExecutorInfos.values + assert(executorInfos.nonEmpty) + executorInfos.foreach { info => + validateLogs("executor", conf, info.logUrlMap) + } + + val driverLogsStatus = listener.driverLogs match { + case Some(logs) => + validateLogs("driver", conf, logs) + "with driver logs" + case None => + "without driver logs" + } + + s"SUCCESS $driverLogsStatus with ${executorInfos.size} executor(s)" + } + } finally sc.stop() + } + + def validateLogs(task: String, conf: SparkConf, logs: collection.Map[String, String]): Unit = { + val expectedLogs = Set("stdout", "stderr") + assert(logs.keySet == expectedLogs, s"Found logs ${logs.keys}") + + val allocId = conf.getenv("NOMAD_ALLOC_ID") + logs.foreach { case (name, urlString) => + val url = new URL(urlString) + assertEquals(url.getProtocol, "http") + assert(url.getPath.matches("/v1/client/fs/logs/[0-9a-f-]{36}")) + assertEquals(url.getQuery, s"follow=true&plain=true&task=$task&type=$name") + } + } +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DiskPersistedRddTestDriver.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DiskPersistedRddTestDriver.scala new file mode 100644 index 0000000000000..24bcdaf361d51 --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/DiskPersistedRddTestDriver.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.nomad + +import scala.util.Random + +import org.apache.spark.SparkContext +import org.apache.spark.storage.StorageLevel + +private[spark] object DiskPersistedRddTestDriver extends TestApplication { + + def main(args: Array[String]): Unit = { + checkArgs(args)("result_url") + val Array(resultUrl) = args + + val sc = new SparkContext() + try { + httpPut(resultUrl) { + sc.parallelize(Seq(Random.nextString(123456), Random.nextString(234567)), 2) + .persist(StorageLevel.DISK_ONLY) + .map(string => string + string) + .persist(StorageLevel.DISK_ONLY) + .groupBy(_.length) // force a shuffle + .persist(StorageLevel.DISK_ONLY) + .map(_._2.map(_.length).sum) + .reduce(_ + _) + .toString + } + } finally sc.stop() + } + +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/EnvironmentPostingTestDriver.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/EnvironmentPostingTestDriver.scala new file mode 100644 index 0000000000000..59edfad37aec4 --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/EnvironmentPostingTestDriver.scala @@ -0,0 +1,52 @@ +/* + * 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.nomad + +import org.apache.spark.{SparkConf, SparkContext} + +private[spark] object EnvironmentPostingTestDriver extends TestApplication { + + def main(args: Array[String]): Unit = { + val rootDriverUrl +: rootExecutorUrl +: environmentVariables = args.toSeq + + val sc = new SparkContext(new SparkConf()) + try { + sc.parallelize(Seq(1)) + .foreach(_ => putEnvironmentVariables(rootExecutorUrl, environmentVariables)) + } catch { case e: Throwable => s"ERROR: $e" } + + try putEnvironmentVariables(rootDriverUrl, environmentVariables) + finally sc.stop() + } + + def putEnvironmentVariables(rootUrl: String, environmentVariables: Seq[String]): Unit = { + + logInfo("---- START ENVIRONMENT VARIABLES ----") + sys.env.foreach { case (k, v) => + logInfo(s"$k=$v") + } + logInfo("---- END ENVIRONMENT VARIABLES ----") + + environmentVariables.foreach { environmentVariable => + httpPut(rootUrl + environmentVariable) { + sys.env.getOrElse(environmentVariable, "NOT SET") + } + } + } + +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/FileDistributionTestDriver.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/FileDistributionTestDriver.scala new file mode 100644 index 0000000000000..5df2f032fec6d --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/FileDistributionTestDriver.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.nomad + +import java.io.File +import java.nio.charset.StandardCharsets + +import com.google.common.io.Files + +import org.apache.spark.{SparkConf, SparkContext} + +private[spark] object FileDistributionTestDriver extends TestApplication { + + def main(args: Array[String]): Unit = { + checkArgs(args)("driver_file", "driver_result_url", "executor_file", "executor_result_url") + val Array(driverFile, driverResultUrl, executorFile, executorResultUrl) = args + + putTestFileContents(driverFile, driverResultUrl) + val sc = new SparkContext(new SparkConf()) + try { + sc.parallelize(Seq(1)).foreach { x => putTestFileContents(executorFile, executorResultUrl) } + } finally { + sc.stop() + } + } + + private def putTestFileContents(path: String, outputUrl: String): Unit = + httpPut(outputUrl) { + Files.toString(new File(path), StandardCharsets.UTF_8) + } + +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/LocalDiskWritingApp.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/LocalDiskWritingApp.scala new file mode 100644 index 0000000000000..8aa6660824e6a --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/LocalDiskWritingApp.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.scheduler.cluster.nomad + +import org.apache.spark.{SparkConf, SparkContext} + +private[spark] object LocalDiskWritingApp extends TestApplication { + + def main(args: Array[String]): Unit = { + checkArgs(args)("driver_result_url", "executor_result_url") + val Array(driverResultUrl, executorResultUrl) = args + + if (driverResultUrl != "NO DRIVER WRITE") { + writeReadAndPutFileContents("DRIVE", driverResultUrl) + } + + val sc = new SparkContext(new SparkConf()) + try { + sc.parallelize(Seq(1)).foreach { x => writeReadAndPutFileContents("EXEC", executorResultUrl) } + } finally { + sc.stop() + } + } + +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ResourceUploaderTestApp.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ResourceUploaderTestApp.scala new file mode 100644 index 0000000000000..53b77e692d784 --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ResourceUploaderTestApp.scala @@ -0,0 +1,48 @@ +/* + * 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.nomad + +import java.nio.charset.StandardCharsets.UTF_8 + +import com.google.common.io.Resources + +import org.apache.spark.SparkContext + +private[spark] object ResourceUploaderTestApp extends TestApplication { + + def main(args: Array[String]): Unit = { + checkArgs(args)("driver_result_url", "executor_result_url") + val Array(driverResultUrl, executorResultUrl) = args + + val sc = new SparkContext + try { + readTestResourceAndUploadTo(driverResultUrl) + sc.parallelize(Seq(1)).foreach { _ => readTestResourceAndUploadTo(executorResultUrl) } + } finally { + sc.stop() + } + } + + private def readTestResourceAndUploadTo(outputUrl: String): Unit = { + httpPut(outputUrl) { + val resource = Thread.currentThread.getContextClassLoader.getResource("test.resource") + Resources.toString(resource, UTF_8) + } + } + +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SaveExecutorInfo.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SaveExecutorInfo.scala new file mode 100644 index 0000000000000..840b4024cc2a1 --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SaveExecutorInfo.scala @@ -0,0 +1,36 @@ +/* + * 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.nomad + +import scala.collection.mutable + +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} +import org.apache.spark.scheduler.cluster.ExecutorInfo + +private[spark] class SaveExecutorInfo extends SparkListener { + val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() + var driverLogs: Option[collection.Map[String, String]] = None + + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + addedExecutorInfos(executor.executorId) = executor.executorInfo + } + + override def onApplicationStart(appStart: SparkListenerApplicationStart): Unit = { + driverLogs = appStart.driverLogs + } +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SaveToDiskTestApp.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SaveToDiskTestApp.scala new file mode 100644 index 0000000000000..baf7ed7d7c45f --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/SaveToDiskTestApp.scala @@ -0,0 +1,39 @@ +/* + * 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.nomad + +import org.apache.spark.SparkContext + +private[spark] object SaveToDiskTestApp extends TestApplication { + + def main(args: Array[String]): Unit = { + checkArgs(args)("driver_result_url", "executor_result_url") + val Array(driverResultUrl, executorResultUrl) = args + + val sc = new SparkContext() + + writeReadAndPutFileContents("CONTENTS WRITTEN TO DRIVER FILE", driverResultUrl) + + sc.parallelize(Seq(1)).foreach { _ => + writeReadAndPutFileContents("CONTENTS WRITTEN TO EXECUTOR FILE", executorResultUrl) + } + + sc.stop() + } + +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/StreamingApp.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/StreamingApp.scala new file mode 100644 index 0000000000000..7105e35dba5e4 --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/StreamingApp.scala @@ -0,0 +1,66 @@ +/* + * 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.nomad + +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable + +import org.apache.spark.SparkConf +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Seconds, StreamingContext} + +private[spark] object StreamingApp extends TestApplication { + + def main(args: Array[String]) { + checkArgs(args)("result_url") + val Array(resultUrl) = args + + val sparkConf = new SparkConf().setAppName("QueueStream") + val ssc = new StreamingContext(sparkConf, Seconds(1)) + try { + httpPut(resultUrl) { + + // Create the queue through which RDDs can be pushed to + // a QueueInputDStream + val rddQueue = new mutable.Queue[RDD[Int]]() + + // Create the QueueInputDStream and use it do some processing + val count = new AtomicInteger() + ssc.queueStream(rddQueue) + .map(x => (x % 10, 1)) + .reduceByKey(_ + _) + .foreachRDD(_ => count.incrementAndGet()) + + ssc.start() + + // Create and push some RDDs into rddQueue + for (i <- 1 to 30) { + rddQueue.synchronized { + rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10) + } + Thread.sleep(1000) + } + + val finalCount = count.get + if (finalCount >= 30) "success" + else finalCount.toString + } + } finally ssc.stop() + } +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/TestApplication.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/TestApplication.scala new file mode 100644 index 0000000000000..d76c23bbcfa36 --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/TestApplication.scala @@ -0,0 +1,101 @@ +/* + * 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.nomad + +import java.io.File +import java.nio.charset.StandardCharsets.UTF_8 + +import com.google.common.io.Files +import org.apache.http.client.methods.HttpPut +import org.apache.http.entity.StringEntity +import org.apache.http.impl.client.HttpClientBuilder + +import org.apache.spark.internal.Logging + +private[spark] trait TestApplication extends Logging { + + def checkArgs(args: Array[String])(expectedArgs: String*): Unit = { + if (args.length != expectedArgs.size) { + usageError(s"expected ${expectedArgs.size} argument(s) but got ${args.length}: $args") { + expectedArgs.map(arg => s"<$arg>").mkString(" ") + } + } + } + + def usageError(m: String)(argsUsage: String): Nothing = { + fatalError( + s"""Error: $m + | + |Usage: ${getClass.getSimpleName} $argsUsage + |""".stripMargin + ) + } + + def fatalError(m: String, ex: Throwable = null): Nothing = { + logError(m, ex) + + // scalastyle:off println + System.out.println(m) + // scalastyle:on println + if (ex != null) { + ex.printStackTrace(System.out) + } + + sys.exit(1) + } + + def httpPut(url: String)(computeContents: => String): Unit = { + try { + logInfo(s"Preparing contents to PUT to $url") + val contentsToPut = + try computeContents + catch { case e: Throwable => s"ERROR computing contents: $e" } + + logInfo(s"Putting to $url: $contentsToPut") + val put = new HttpPut(url) + put.setEntity(new StringEntity(contentsToPut, UTF_8)) + val http = HttpClientBuilder.create().build() + try { + val response = http.execute(put) + if (response.getStatusLine.getStatusCode / 100 != 2) { + sys.error(s"Got non-2xx response: $response") + } + logInfo(s"PUT to $url complete") + } finally http.close() + } catch { + case e: Throwable => + throw new RuntimeException(s"ERROR putting to $url: $e", e) + } + } + + def writeReadAndPutFileContents(contents: String, outputUrl: String): Unit = + httpPut(outputUrl) { + val dir = new File("target") + dir.mkdir() + val file = new File(dir, "temp.txt") + Files.write(contents, file, UTF_8) + Files.toString(file, UTF_8) + } + + def assertEquals[A](a: A, b: A): Unit = { + if (a != b) { + throw new RuntimeException(s"$a is not equal to $b") + } + } + +} diff --git a/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ThrowExceptionAfterContextInit.scala b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ThrowExceptionAfterContextInit.scala new file mode 100644 index 0000000000000..4604a23cea250 --- /dev/null +++ b/resource-managers/nomad/test-apps/src/main/scala/org/apache/spark/scheduler/cluster/nomad/ThrowExceptionAfterContextInit.scala @@ -0,0 +1,32 @@ +/* + * 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.nomad + +import org.apache.spark.{SparkConf, SparkContext} + +private object ThrowExceptionAfterContextInit extends TestApplication { + def main(args: Array[String]): Unit = { + checkArgs(args)() + + val sc = new SparkContext(new SparkConf() + .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) + .setAppName("yarn test with failure")) + + throw new Exception("exception after sc initialized") + } +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index cbc6e60e839c1..57727c0d0d1bd 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -49,6 +49,9 @@ private[spark] abstract class YarnSchedulerBackend( protected var totalExpectedExecutors = 0 + // The num of current max ExecutorId used to re-register appMaster + @volatile protected var currentExecutorIdCounter = 0 + private val yarnSchedulerEndpoint = new YarnSchedulerEndpoint(rpcEnv) private val yarnSchedulerEndpointRef = rpcEnv.setupEndpoint( @@ -193,6 +196,11 @@ private[spark] abstract class YarnSchedulerBackend( private class YarnDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) extends DriverEndpoint(rpcEnv, sparkProperties) { + override protected def synchronizedOnNewExecutorId(executorId: String): Unit = + if (currentExecutorIdCounter < executorId.toInt) { + currentExecutorIdCounter = executorId.toInt + } + /** * When onDisconnected is received at the driver endpoint, the superclass DriverEndpoint * handles it by assuming the Executor was lost for a bad reason and removes the executor