From c5967310740e8a71c6b9d7ed4d7cc413582268c7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 27 Sep 2023 21:22:07 -0500 Subject: [PATCH] [SPARK-44442][MESOS] Remove Mesos support ### What changes were proposed in this pull request? Remove Mesos resource manager support and docs, previously deprecated in Spark 3.2, for Spark 4.0. ### Why are the changes needed? Mesos is no longer supported and has been deprecated for several versions. ### Does this PR introduce _any_ user-facing change? Other than the lack of Mesos support, no ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43135 from srowen/SPARK-44442. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .github/labeler.yml | 3 - .github/workflows/benchmark.yml | 2 +- .github/workflows/build_and_test.yml | 4 +- .github/workflows/maven_test.yml | 12 +- LICENSE-binary | 3 +- NOTICE-binary | 3 - R/pkg/tests/fulltests/test_sparkR.R | 6 +- README.md | 2 +- assembly/pom.xml | 10 - .../protocol/BlockTransferMessage.java | 4 - .../protocol/mesos/RegisterDriver.java | 77 -- .../mesos/ShuffleServiceHeartbeat.java | 53 - conf/spark-env.sh.template | 1 - .../scala/org/apache/spark/SparkConf.scala | 2 +- .../scala/org/apache/spark/SparkContext.scala | 14 +- .../spark/api/java/JavaSparkContext.scala | 10 +- .../apache/spark/deploy/PythonRunner.scala | 1 - .../org/apache/spark/deploy/SparkSubmit.scala | 73 +- .../spark/deploy/SparkSubmitArguments.scala | 8 +- .../spark/deploy/history/HistoryServer.scala | 2 +- .../deploy/rest/RestSubmissionClient.scala | 4 +- .../apache/spark/deploy/security/README.md | 2 +- .../org/apache/spark/executor/Executor.scala | 5 +- .../spark/internal/config/package.scala | 9 +- .../apache/spark/metrics/MetricsSystem.scala | 3 - .../spark/resource/ResourceProfile.scala | 4 +- .../spark/scheduler/SchedulerBackend.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 1 - .../CoarseGrainedSchedulerBackend.scala | 5 +- .../scala/org/apache/spark/util/Utils.scala | 13 +- .../apache/spark/SecurityManagerSuite.scala | 2 +- .../spark/deploy/SparkSubmitSuite.scala | 22 - .../rest/StandaloneRestSubmitSuite.scala | 6 - dev/create-release/release-build.sh | 2 +- dev/create-release/releaseutils.py | 1 - dev/deps/spark-deps-hadoop-3-hive-2.3 | 1 - dev/lint-java | 2 +- dev/mima | 2 +- dev/sbt-checkstyle | 2 +- dev/scalastyle | 2 +- dev/sparktestsupport/modules.py | 8 - dev/test-dependencies.sh | 2 +- docs/_config.yml | 3 +- docs/_layouts/global.html | 1 - docs/building-spark.md | 6 +- docs/cluster-overview.md | 8 +- docs/configuration.md | 34 +- docs/core-migration-guide.md | 2 + docs/hardware-provisioning.md | 3 +- docs/index.md | 3 - docs/job-scheduling.md | 23 +- docs/monitoring.md | 8 - docs/rdd-programming-guide.md | 2 +- docs/running-on-mesos.md | 901 ----------------- docs/security.md | 26 +- docs/spark-standalone.md | 2 +- docs/streaming-programming-guide.md | 21 +- docs/submitting-applications.md | 16 - .../launcher/AbstractCommandBuilder.java | 1 - .../launcher/SparkClassCommandBuilder.java | 13 +- .../SparkSubmitCommandBuilderSuite.java | 4 - pom.xml | 7 - project/SparkBuild.scala | 4 +- python/README.md | 2 +- .../source/user_guide/python_packaging.rst | 2 +- python/pyspark/context.py | 2 +- .../org/apache/spark/repl/ReplSuite.scala | 24 - .../k8s/features/LocalDirsFeatureStep.scala | 2 +- resource-managers/mesos/pom.xml | 128 --- .../mesos/MesosExternalBlockStoreClient.java | 124 --- ...che.spark.scheduler.ExternalClusterManager | 18 - .../deploy/mesos/MesosClusterDispatcher.scala | 136 --- .../MesosClusterDispatcherArguments.scala | 149 --- .../deploy/mesos/MesosDriverDescription.scala | 70 -- .../mesos/MesosExternalShuffleService.scala | 131 --- .../apache/spark/deploy/mesos/config.scala | 420 -------- .../spark/deploy/mesos/ui/DriverPage.scala | 186 ---- .../deploy/mesos/ui/MesosClusterPage.scala | 156 --- .../deploy/mesos/ui/MesosClusterUI.scala | 48 - .../deploy/rest/mesos/MesosRestServer.scala | 176 ---- .../spark/executor/MesosExecutorBackend.scala | 132 --- .../cluster/mesos/MesosClusterManager.scala | 65 -- .../mesos/MesosClusterPersistenceEngine.scala | 135 --- .../cluster/mesos/MesosClusterScheduler.scala | 904 ------------------ .../mesos/MesosClusterSchedulerSource.scala | 41 - .../MesosCoarseGrainedSchedulerBackend.scala | 820 ---------------- .../MesosFineGrainedSchedulerBackend.scala | 468 --------- .../cluster/mesos/MesosProtoUtils.scala | 53 - .../cluster/mesos/MesosScheduler.scala | 28 - .../mesos/MesosSchedulerBackendUtil.scala | 307 ------ .../cluster/mesos/MesosSchedulerUtils.scala | 621 ------------ .../src/test/resources/log4j2.properties | 31 - ...MesosClusterDispatcherArgumentsSuite.scala | 63 -- .../mesos/MesosClusterDispatcherSuite.scala | 40 - .../rest/mesos/MesosRestServerSuite.scala | 91 -- .../mesos/MesosClusterManagerSuite.scala | 57 -- .../mesos/MesosClusterSchedulerSuite.scala | 836 ---------------- ...osCoarseGrainedSchedulerBackendSuite.scala | 843 ---------------- ...esosFineGrainedSchedulerBackendSuite.scala | 420 -------- .../cluster/mesos/MesosProtoUtilsSuite.scala | 48 - .../MesosSchedulerBackendUtilSuite.scala | 71 -- .../mesos/MesosSchedulerUtilsSuite.scala | 398 -------- .../spark/scheduler/cluster/mesos/Utils.scala | 231 ----- sbin/start-mesos-dispatcher.sh | 51 - sbin/start-mesos-shuffle-service.sh | 36 - sbin/stop-mesos-dispatcher.sh | 33 - sbin/stop-mesos-shuffle-service.sh | 26 - .../apache/spark/streaming/Checkpoint.scala | 3 +- .../spark/streaming/StreamingContext.scala | 2 +- 110 files changed, 117 insertions(+), 9991 deletions(-) delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/ShuffleServiceHeartbeat.java delete mode 100644 docs/running-on-mesos.md delete mode 100644 resource-managers/mesos/pom.xml delete mode 100644 resource-managers/mesos/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalBlockStoreClient.java delete mode 100644 resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosProtoUtils.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosScheduler.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala delete mode 100644 resource-managers/mesos/src/test/resources/log4j2.properties delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosProtoUtilsSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala delete mode 100755 sbin/start-mesos-dispatcher.sh delete mode 100755 sbin/start-mesos-shuffle-service.sh delete mode 100755 sbin/stop-mesos-dispatcher.sh delete mode 100755 sbin/stop-mesos-shuffle-service.sh diff --git a/.github/labeler.yml b/.github/labeler.yml index b252edd88739d..f21b90d460fb6 100644 --- a/.github/labeler.yml +++ b/.github/labeler.yml @@ -136,9 +136,6 @@ R: - "bin/sparkR*" YARN: - "resource-managers/yarn/**/*" -MESOS: - - "resource-managers/mesos/**/*" - - "sbin/*mesos*.sh" KUBERNETES: - "resource-managers/kubernetes/**/*" WINDOWS: diff --git a/.github/workflows/benchmark.yml b/.github/workflows/benchmark.yml index 11ebb8ae7e2ad..f4b17b4117a44 100644 --- a/.github/workflows/benchmark.yml +++ b/.github/workflows/benchmark.yml @@ -170,7 +170,7 @@ jobs: key: tpcds-${{ hashFiles('.github/workflows/benchmark.yml', 'sql/core/src/test/scala/org/apache/spark/sql/TPCDSSchema.scala') }} - name: Run benchmarks run: | - ./build/sbt -Pscala-${{ github.event.inputs.scala }} -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pspark-ganglia-lgpl Test/package + ./build/sbt -Pscala-${{ github.event.inputs.scala }} -Pyarn -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pspark-ganglia-lgpl Test/package # Make less noisy cp conf/log4j2.properties.template conf/log4j2.properties sed -i 's/rootLogger.level = info/rootLogger.level = warn/g' conf/log4j2.properties diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 393183650a573..89da6fcffe467 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -149,7 +149,7 @@ jobs: - >- streaming, sql-kafka-0-10, streaming-kafka-0-10, mllib-local, mllib, - yarn, mesos, kubernetes, hadoop-cloud, spark-ganglia-lgpl, + yarn, kubernetes, hadoop-cloud, spark-ganglia-lgpl, connect, protobuf # Here, we split Hive and SQL tests into some of slow ones and the rest of them. included-tags: [""] @@ -823,7 +823,7 @@ jobs: export MAVEN_CLI_OPTS="--no-transfer-progress" export JAVA_VERSION=${{ matrix.java }} # It uses Maven's 'install' intentionally, see https://github.com/apache/spark/pull/26414. - ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Djava.version=${JAVA_VERSION/-ea} install + ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Djava.version=${JAVA_VERSION/-ea} install rm -rf ~/.m2/repository/org/apache/spark # Any TPC-DS related updates on this job need to be applied to tpcds-1g-gen job of benchmark.yml as well diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 9b6c583bd7ba0..169def690ae49 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -65,7 +65,7 @@ jobs: - >- connector#kafka-0-10,connector#kafka-0-10-sql,connector#kafka-0-10-token-provider,connector#spark-ganglia-lgpl,connector#protobuf,connector#avro - >- - sql#catalyst,resource-managers#yarn,resource-managers#mesos,resource-managers#kubernetes#core + sql#catalyst,resource-managers#yarn,resource-managers#kubernetes#core - >- connect # Here, we split Hive and SQL tests into some of slow ones and the rest of them. @@ -180,18 +180,18 @@ jobs: export JAVA_VERSION=${{ matrix.java }} # Replace with the real module name, for example, connector#kafka-0-10 -> connector/kafka-0-10 export TEST_MODULES=`echo "$MODULES_TO_TEST" | sed -e "s%#%/%g"` - ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} clean install + ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} clean install if [[ "$INCLUDED_TAGS" != "" ]]; then - ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pmesos -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} -Dtest.include.tags="$INCLUDED_TAGS" test -fae + ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} -Dtest.include.tags="$INCLUDED_TAGS" test -fae elif [[ "$EXCLUDED_TAGS" != "" ]]; then - ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pmesos -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} -Dtest.exclude.tags="$EXCLUDED_TAGS" test -fae + ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} -Dtest.exclude.tags="$EXCLUDED_TAGS" test -fae elif [[ "$MODULES_TO_TEST" == "connect" ]]; then ./build/mvn $MAVEN_CLI_OPTS -Djava.version=${JAVA_VERSION/-ea} -pl connector/connect/client/jvm,connector/connect/common,connector/connect/server test -fae elif [[ "$MODULES_TO_TEST" == *"sql#hive-thriftserver"* ]]; then # To avoid a compilation loop, for the `sql/hive-thriftserver` module, run `clean install` instead - ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pmesos -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} clean install -fae + ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} clean install -fae else - ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pmesos -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Pspark-ganglia-lgpl -Phadoop-cloud -Djava.version=${JAVA_VERSION/-ea} test -fae + ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Pspark-ganglia-lgpl -Phadoop-cloud -Djava.version=${JAVA_VERSION/-ea} test -fae fi - name: Clean up local Maven repository run: | diff --git a/LICENSE-binary b/LICENSE-binary index 865a30f0f3089..f0f59e7d57644 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -348,8 +348,7 @@ org.apache.directory.server:apacheds-i18n org.apache.directory.server:apacheds-kerberos-codec org.apache.htrace:htrace-core org.apache.ivy:ivy -org.apache.mesos:mesos -org.apache.parquet:parquet-column +=org.apache.parquet:parquet-column org.apache.parquet:parquet-common org.apache.parquet:parquet-encoding org.apache.parquet:parquet-format diff --git a/NOTICE-binary b/NOTICE-binary index 95653c6f49a07..ef2dba45055a0 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -673,9 +673,6 @@ Copyright 2002-2012 The Apache Software Foundation Google Guice - Core Library Copyright 2006-2011 Google, Inc. -mesos -Copyright 2017 The Apache Software Foundation - Apache Parquet Hadoop Bundle (Incubating) Copyright 2015 The Apache Software Foundation diff --git a/R/pkg/tests/fulltests/test_sparkR.R b/R/pkg/tests/fulltests/test_sparkR.R index 4232f5ec430f6..2044df7983988 100644 --- a/R/pkg/tests/fulltests/test_sparkR.R +++ b/R/pkg/tests/fulltests/test_sparkR.R @@ -18,7 +18,7 @@ context("functions in sparkR.R") test_that("sparkCheckInstall", { - # "local, yarn-client, mesos-client" mode, SPARK_HOME was set correctly, + # "local, yarn-client" mode, SPARK_HOME was set correctly, # and the SparkR job was submitted by "spark-submit" sparkHome <- paste0(tempdir(), "/", "sparkHome") dir.create(sparkHome) @@ -27,14 +27,14 @@ test_that("sparkCheckInstall", { expect_true(is.null(sparkCheckInstall(sparkHome, master, deployMode))) unlink(sparkHome, recursive = TRUE) - # "yarn-cluster, mesos-cluster" mode, SPARK_HOME was not set, + # "yarn-cluster" mode, SPARK_HOME was not set, # and the SparkR job was submitted by "spark-submit" sparkHome <- "" master <- "" deployMode <- "" expect_true(is.null(sparkCheckInstall(sparkHome, master, deployMode))) - # "yarn-client, mesos-client" mode, SPARK_HOME was not set + # "yarn-client" mode, SPARK_HOME was not set sparkHome <- "" master <- "yarn" deployMode <- "client" diff --git a/README.md b/README.md index 310df41f4654b..ec29a1c49b412 100644 --- a/README.md +++ b/README.md @@ -77,7 +77,7 @@ To run one of them, use `./bin/run-example [params]`. For example: will run the Pi example locally. You can set the MASTER environment variable when running examples to submit -examples to a cluster. This can be a mesos:// or spark:// URL, +examples to a cluster. This can be spark:// URL, "yarn" to run on YARN, and "local" to run locally with one thread, or "local[N]" to run locally with N threads. You can also use an abbreviated class name if the class is in the `examples` diff --git a/assembly/pom.xml b/assembly/pom.xml index 020ef80c4ad3e..48fe8f46341e6 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -142,16 +142,6 @@ - - mesos - - - org.apache.spark - spark-mesos_${scala.binary.version} - ${project.version} - - - connect diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index 33411baa09f8a..80143c591d909 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -24,8 +24,6 @@ import org.apache.spark.network.protocol.Encodable; import org.apache.spark.network.shuffle.ExternalBlockHandler; -import org.apache.spark.network.shuffle.protocol.mesos.RegisterDriver; -import org.apache.spark.network.shuffle.protocol.mesos.ShuffleServiceHeartbeat; /** * Messages handled by the {@link ExternalBlockHandler}, or @@ -73,8 +71,6 @@ public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { case 1: return UploadBlock.decode(buf); case 2: return RegisterExecutor.decode(buf); case 3: return StreamHandle.decode(buf); - case 4: return RegisterDriver.decode(buf); - case 5: return ShuffleServiceHeartbeat.decode(buf); case 6: return UploadBlockStream.decode(buf); case 7: return RemoveBlocks.decode(buf); case 8: return BlocksRemoved.decode(buf); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java deleted file mode 100644 index d5f53ccb7f741..0000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.shuffle.protocol.mesos; - -import com.google.common.base.Objects; -import io.netty.buffer.ByteBuf; - -import org.apache.spark.network.protocol.Encoders; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; - -// Needed by ScalaDoc. See SPARK-7726 -import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; - -/** - * A message sent from the driver to register with the MesosExternalShuffleService. - */ -public class RegisterDriver extends BlockTransferMessage { - private final String appId; - private final long heartbeatTimeoutMs; - - public RegisterDriver(String appId, long heartbeatTimeoutMs) { - this.appId = appId; - this.heartbeatTimeoutMs = heartbeatTimeoutMs; - } - - public String getAppId() { return appId; } - - public long getHeartbeatTimeoutMs() { return heartbeatTimeoutMs; } - - @Override - protected Type type() { return Type.REGISTER_DRIVER; } - - @Override - public int encodedLength() { - return Encoders.Strings.encodedLength(appId) + Long.SIZE / Byte.SIZE; - } - - @Override - public void encode(ByteBuf buf) { - Encoders.Strings.encode(buf, appId); - buf.writeLong(heartbeatTimeoutMs); - } - - @Override - public int hashCode() { - return Objects.hashCode(appId, heartbeatTimeoutMs); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof RegisterDriver)) { - return false; - } - return Objects.equal(appId, ((RegisterDriver) o).appId); - } - - public static RegisterDriver decode(ByteBuf buf) { - String appId = Encoders.Strings.decode(buf); - long heartbeatTimeout = buf.readLong(); - return new RegisterDriver(appId, heartbeatTimeout); - } -} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/ShuffleServiceHeartbeat.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/ShuffleServiceHeartbeat.java deleted file mode 100644 index b30bb9aed55b6..0000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/ShuffleServiceHeartbeat.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.shuffle.protocol.mesos; - -import io.netty.buffer.ByteBuf; -import org.apache.spark.network.protocol.Encoders; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; - -// Needed by ScalaDoc. See SPARK-7726 -import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; - -/** - * A heartbeat sent from the driver to the MesosExternalShuffleService. - */ -public class ShuffleServiceHeartbeat extends BlockTransferMessage { - private final String appId; - - public ShuffleServiceHeartbeat(String appId) { - this.appId = appId; - } - - public String getAppId() { return appId; } - - @Override - protected Type type() { return Type.HEARTBEAT; } - - @Override - public int encodedLength() { return Encoders.Strings.encodedLength(appId); } - - @Override - public void encode(ByteBuf buf) { - Encoders.Strings.encode(buf, appId); - } - - public static ShuffleServiceHeartbeat decode(ByteBuf buf) { - return new ShuffleServiceHeartbeat(Encoders.Strings.decode(buf)); - } -} diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index e9491995e727e..788341c72c734 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -30,7 +30,6 @@ # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program # - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data -# - MESOS_NATIVE_JAVA_LIBRARY, to point to your libmesos.so if you use Mesos # Options read in any mode # - SPARK_CONF_DIR, Alternate conf dir. (Default: ${SPARK_HOME}/conf) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0953ffe5ff80b..091413bb9cc92 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -498,7 +498,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria private[spark] def validateSettings(): Unit = { if (contains("spark.local.dir")) { val msg = "Note that spark.local.dir will be overridden by the value set by " + - "the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone/kubernetes and LOCAL_DIRS" + + "the cluster manager (via SPARK_LOCAL_DIRS in standalone/kubernetes and LOCAL_DIRS" + " in YARN)." logWarning(msg) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 277ff6364d4f7..dd95b71d4d598 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -132,7 +132,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Alternative constructor that allows setting common Spark properties directly * - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters */ @@ -142,7 +142,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Alternative constructor that allows setting common Spark properties directly * - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI. * @param sparkHome Location where Spark is installed on cluster nodes. * @param jars Collection of JARs to send to the cluster. These can be paths on the local file @@ -164,7 +164,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Alternative constructor that allows setting common Spark properties directly * - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI. */ private[spark] def this(master: String, appName: String) = @@ -173,7 +173,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Alternative constructor that allows setting common Spark properties directly * - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI. * @param sparkHome Location where Spark is installed on cluster nodes. */ @@ -183,7 +183,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Alternative constructor that allows setting common Spark properties directly * - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI. * @param sparkHome Location where Spark is installed on cluster nodes. * @param jars Collection of JARs to send to the cluster. These can be paths on the local file @@ -352,7 +352,6 @@ class SparkContext(config: SparkConf) extends Logging { * (i.e. * in case of local spark app something like 'local-1433865536131' * in case of YARN something like 'application_1433865536131_34483' - * in case of MESOS something like 'driver-20170926223339-0001' * ) */ def applicationId: String = _applicationId @@ -557,9 +556,6 @@ class SparkContext(config: SparkConf) extends Logging { Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => executorEnvs("SPARK_PREPEND_CLASSES") = v } - // The Mesos scheduler backend relies on this environment variable to set executor memory. - // TODO: Set this only in the Mesos scheduler. - executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" executorEnvs ++= _conf.getExecutorEnv executorEnvs("SPARK_USER") = sparkUser diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 80d56e4fac352..80c6e1f5cb81e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -58,13 +58,13 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { def this(conf: SparkConf) = this(new SparkContext(conf)) /** - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI */ def this(master: String, appName: String) = this(new SparkContext(master, appName)) /** - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters */ @@ -72,7 +72,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { this(conf.setMaster(master).setAppName(appName)) /** - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI * @param sparkHome The SPARK_HOME directory on the worker nodes * @param jarFile JAR file to send to the cluster. This can be a path on the local file system @@ -82,7 +82,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { this(new SparkContext(master, appName, sparkHome, Seq(jarFile))) /** - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI * @param sparkHome The SPARK_HOME directory on the worker nodes * @param jars Collection of JARs to send to the cluster. These can be paths on the local file @@ -92,7 +92,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { this(new SparkContext(master, appName, sparkHome, jars.toSeq)) /** - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI * @param sparkHome The SPARK_HOME directory on the worker nodes * @param jars Collection of JARs to send to the cluster. These can be paths on the local file diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index 15a402d3cc9df..667bf8bbc9754 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -86,7 +86,6 @@ object PythonRunner { sys.env.get("PYTHONHASHSEED").foreach(env.put("PYTHONHASHSEED", _)) // if OMP_NUM_THREADS is not explicitly set, override it with the number of cores if (sparkConf.getOption("spark.yarn.appMasterEnv.OMP_NUM_THREADS").isEmpty && - sparkConf.getOption("spark.mesos.driverEnv.OMP_NUM_THREADS").isEmpty && sparkConf.getOption("spark.kubernetes.driverEnv.OMP_NUM_THREADS").isEmpty) { // SPARK-28843: limit the OpenMP thread pool to the number of cores assigned to the driver // this avoids high memory consumption with pandas/numpy because of a large OpenMP thread pool 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 d5947649592ea..f101a180ea5a3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -60,7 +60,7 @@ import org.apache.spark.util._ /** * Whether to submit, kill, or request the status of an application. - * The latter two operations are currently supported only for standalone and Mesos cluster modes. + * The latter two operations are currently supported only for standalone cluster mode. */ private[deploy] object SparkSubmitAction extends Enumeration { type SparkSubmitAction = Value @@ -249,11 +249,10 @@ private[spark] class SparkSubmit extends Logging { v match { case "yarn" => YARN case m if m.startsWith("spark") => STANDALONE - case m if m.startsWith("mesos") => MESOS case m if m.startsWith("k8s") => KUBERNETES case m if m.startsWith("local") => LOCAL case _ => - error("Master must either be yarn or start with spark, mesos, k8s, or local") + error("Master must either be yarn or start with spark, k8s, or local") -1 } case None => LOCAL // default master or remote mode. @@ -315,7 +314,6 @@ private[spark] class SparkSubmit extends Logging { case _ => } val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER - val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER val isStandAloneCluster = clusterManager == STANDALONE && deployMode == CLUSTER val isKubernetesCluster = clusterManager == KUBERNETES && deployMode == CLUSTER val isKubernetesClient = clusterManager == KUBERNETES && deployMode == CLIENT @@ -324,9 +322,9 @@ private[spark] class SparkSubmit extends Logging { val isCustomClasspathInClusterModeDisallowed = !sparkConf.get(ALLOW_CUSTOM_CLASSPATH_BY_PROXY_USER_IN_CLUSTER_MODE) && args.proxyUser != null && - (isYarnCluster || isMesosCluster || isStandAloneCluster || isKubernetesCluster) + (isYarnCluster || isStandAloneCluster || isKubernetesCluster) - if (!isMesosCluster && !isStandAloneCluster) { + if (!isStandAloneCluster) { // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files // too for packages that include Python code val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies( @@ -371,10 +369,8 @@ private[spark] class SparkSubmit extends Logging { val hadoopConf = conf.getOrElse(SparkHadoopUtil.newConfiguration(sparkConf)) val targetDir = Utils.createTempDir() - // Kerberos is not supported in standalone mode, and keytab support is not yet available - // in Mesos cluster mode. + // Kerberos is not supported in standalone mode if (clusterManager != STANDALONE - && !isMesosCluster && args.principal != null && args.keytab != null) { // If client mode, make sure the keytab is just a local path. @@ -594,11 +590,6 @@ private[spark] class SparkSubmit extends Logging { error("Distributing R packages with standalone cluster is not supported.") } - // TODO: Support distributing R packages with mesos cluster - if (args.isR && clusterManager == MESOS && !RUtils.rPackages.isEmpty) { - error("Distributing R packages with mesos 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) { @@ -656,13 +647,13 @@ private[spark] class SparkSubmit extends Logging { OptionAssigner(args.pyFiles, ALL_CLUSTER_MGRS, CLUSTER, confKey = SUBMIT_PYTHON_FILES.key), // Propagate attributes for dependency resolution at the driver side - OptionAssigner(args.packages, STANDALONE | MESOS | KUBERNETES, + OptionAssigner(args.packages, STANDALONE | KUBERNETES, CLUSTER, confKey = JAR_PACKAGES.key), - OptionAssigner(args.repositories, STANDALONE | MESOS | KUBERNETES, + OptionAssigner(args.repositories, STANDALONE | KUBERNETES, CLUSTER, confKey = JAR_REPOSITORIES.key), - OptionAssigner(args.ivyRepoPath, STANDALONE | MESOS | KUBERNETES, + OptionAssigner(args.ivyRepoPath, STANDALONE | KUBERNETES, CLUSTER, confKey = JAR_IVY_REPO_PATH.key), - OptionAssigner(args.packagesExclusions, STANDALONE | MESOS | KUBERNETES, + OptionAssigner(args.packagesExclusions, STANDALONE | KUBERNETES, CLUSTER, confKey = JAR_PACKAGES_EXCLUSIONS.key), // Yarn only @@ -681,22 +672,22 @@ private[spark] class SparkSubmit extends Logging { confKey = EXECUTOR_INSTANCES.key), OptionAssigner(args.executorCores, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES, confKey = EXECUTOR_CORES.key), - OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | KUBERNETES, ALL_DEPLOY_MODES, + OptionAssigner(args.executorMemory, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES, confKey = EXECUTOR_MEMORY.key), - OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, + OptionAssigner(args.totalExecutorCores, STANDALONE | KUBERNETES, ALL_DEPLOY_MODES, confKey = CORES_MAX.key), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, + OptionAssigner(args.files, LOCAL | STANDALONE | KUBERNETES, ALL_DEPLOY_MODES, confKey = FILES.key), - OptionAssigner(args.archives, LOCAL | STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, + OptionAssigner(args.archives, LOCAL | STANDALONE | KUBERNETES, ALL_DEPLOY_MODES, confKey = ARCHIVES.key), OptionAssigner(args.jars, LOCAL, CLIENT, confKey = JARS.key), - OptionAssigner(args.jars, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, + OptionAssigner(args.jars, STANDALONE | KUBERNETES, ALL_DEPLOY_MODES, confKey = JARS.key), - OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, + OptionAssigner(args.driverMemory, STANDALONE | YARN | KUBERNETES, CLUSTER, confKey = DRIVER_MEMORY.key), - OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, + OptionAssigner(args.driverCores, STANDALONE | YARN | KUBERNETES, CLUSTER, confKey = DRIVER_CORES.key), - OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, + OptionAssigner(args.supervise.toString, STANDALONE, CLUSTER, confKey = DRIVER_SUPERVISE.key), OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, confKey = JAR_IVY_REPO_PATH.key), @@ -790,8 +781,7 @@ private[spark] class SparkSubmit extends Logging { } } - if ((clusterManager == MESOS || clusterManager == KUBERNETES) - && UserGroupInformation.isSecurityEnabled) { + if (clusterManager == KUBERNETES && UserGroupInformation.isSecurityEnabled) { setRMPrincipal(sparkConf) } @@ -816,26 +806,6 @@ private[spark] class SparkSubmit extends Logging { } } - if (isMesosCluster) { - assert(args.useRest, "Mesos cluster mode is only supported through the REST submission API") - childMainClass = REST_CLUSTER_SUBMIT_CLASS - if (args.isPython) { - // Second argument is main class - childArgs += (args.primaryResource, "") - if (args.pyFiles != null) { - sparkConf.set(SUBMIT_PYTHON_FILES, args.pyFiles.split(",").toSeq) - } - } else if (args.isR) { - // Second argument is main class - childArgs += (args.primaryResource, "") - } else { - childArgs += (args.primaryResource, args.mainClass) - } - if (args.childArgs != null) { - childArgs ++= args.childArgs - } - } - if (isKubernetesCluster) { childMainClass = KUBERNETES_CLUSTER_SUBMIT_CLASS if (args.primaryResource != SparkLauncher.NO_RESOURCE) { @@ -897,7 +867,7 @@ private[spark] class SparkSubmit extends Logging { val formattedPyFiles = if (deployMode != CLUSTER) { PythonRunner.formatPaths(resolvedPyFiles).mkString(",") } else { - // Ignoring formatting python path in yarn and mesos cluster mode, these two modes + // Ignoring formatting python path in yarn cluster mode, these two modes // support dealing with remote python files, they could distribute and add python files // locally. resolvedPyFiles @@ -926,7 +896,7 @@ private[spark] class SparkSubmit extends Logging { } // [SPARK-20328]. HadoopRDD calls into a Hadoop library that fetches delegation tokens with - // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos or Kubernetes + // renewer set to the YARN ResourceManager. Since YARN isn't configured in Kubernetes // mode, we must trick it into thinking we're YARN. private def setRMPrincipal(sparkConf: SparkConf): Unit = { val shortUserName = UserGroupInformation.getCurrentUser.getShortUserName @@ -1066,10 +1036,9 @@ object SparkSubmit extends CommandLineUtils with Logging { // Cluster managers private val YARN = 1 private val STANDALONE = 2 - private val MESOS = 4 private val LOCAL = 8 private val KUBERNETES = 16 - private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL | KUBERNETES + private val ALL_CLUSTER_MGRS = YARN | STANDALONE | LOCAL | KUBERNETES // Deploy modes private val CLIENT = 1 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 c0a6b03d22fa2..c100e92a1a3bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -502,7 +502,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S logInfo( s""" |Options: - | --master MASTER_URL spark://host:port, mesos://host:port, yarn, + | --master MASTER_URL spark://host:port, yarn, | k8s://https://host:port, or local (Default: local[*]). | --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or | on one of the worker machines inside the cluster ("cluster") @@ -559,14 +559,14 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --driver-cores NUM Number of cores used by the driver, only in cluster mode | (Default: 1). | - | Spark standalone or Mesos with cluster deploy mode only: + | Spark standalone with cluster deploy mode only: | --supervise If given, restarts the driver on failure. | - | Spark standalone, Mesos or K8s with cluster deploy mode only: + | Spark standalone or K8s with cluster deploy mode only: | --kill SUBMISSION_ID If given, kills the driver specified. | --status SUBMISSION_ID If given, requests the status of the driver specified. | - | Spark standalone, Mesos and Kubernetes only: + | Spark standalone and Kubernetes only: | --total-executor-cores NUM Total cores for all executors. | | Spark standalone, YARN and Kubernetes only: diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index bea3f9ec84a4b..875e34041d3b8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -40,7 +40,7 @@ import org.apache.spark.util.{ShutdownHookManager, SystemClock, Utils} * A web server that renders SparkUIs of completed applications. * * For the standalone mode, MasterWebUI already achieves this functionality. Thus, the - * main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos). + * main use case of the HistoryServer is in other deploy modes (e.g. Yarn). * * The logging directory structure is as follows: Within the given base directory, each * application's event logs are maintained in the application's own sub-directory. This diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 8a0fc886e60ca..a690937c2020c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -409,7 +409,7 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { private[spark] object RestSubmissionClient { - val supportedMasterPrefixes = Seq("spark://", "mesos://") + val supportedMasterPrefixes = Seq("spark://") // SPARK_HOME and SPARK_CONF_DIR are filtered out because they are usually wrong // on the remote machine (SPARK-12345) (SPARK-25934) @@ -423,7 +423,7 @@ private[spark] object RestSubmissionClient { */ private[rest] def filterSystemEnvironment(env: Map[String, String]): Map[String, String] = { env.filterKeys { k => - (k.startsWith("SPARK_") && !EXCLUDED_SPARK_ENV_VARS.contains(k)) || k.startsWith("MESOS_") + k.startsWith("SPARK_") && !EXCLUDED_SPARK_ENV_VARS.contains(k) }.toMap } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/README.md b/core/src/main/scala/org/apache/spark/deploy/security/README.md index 0d98ce43919e6..c71a6e8e199dd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/README.md +++ b/core/src/main/scala/org/apache/spark/deploy/security/README.md @@ -182,7 +182,7 @@ tokens at every renewal interval instead, and distribute those tokens to executo This also has another advantage on top of supporting services like HBase: it removes the dependency on an external renewal service (like YARN). That way, Spark's renewal feature can be used with -resource managers that are not DT-aware, such as Mesos or Kubernetes, as long as the application +resource managers that are not DT-aware, such as Kubernetes, as long as the application has access to a keytab. diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 6b1e07151b374..5f861480c3eba 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -67,9 +67,8 @@ private[spark] class IsolatedSessionState( /** * Spark executor, backed by a threadpool to run tasks. * - * This can be used with Mesos, YARN, kubernetes and the standalone scheduler. - * An internal RPC interface is used for communication with the driver, - * except in the case of Mesos fine-grained mode. + * This can be used with YARN, kubernetes and the standalone scheduler. + * An internal RPC interface is used for communication with the driver. */ private[spark] class Executor( executorId: String, diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 2dcd3af7a52dd..2ae52ce8d6cb8 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -357,11 +357,10 @@ package object config { .createWithDefault(0.1) private[spark] val CORES_MAX = ConfigBuilder("spark.cores.max") - .doc("When running on a standalone deploy cluster or a Mesos cluster in coarse-grained " + - "sharing mode, the maximum amount of CPU cores to request for the application from across " + + .doc("When running on a standalone deploy cluster, " + + "the maximum amount of CPU cores to request for the application from across " + "the cluster (not from each machine). If not set, the default will be " + - "`spark.deploy.defaultCores` on Spark's standalone cluster manager, or infinite " + - "(all available cores) on Mesos.") + "`spark.deploy.defaultCores` on Spark's standalone cluster manager") .version("0.6.0") .intConf .createOptional @@ -1077,7 +1076,7 @@ package object config { private[spark] val DRIVER_SUPERVISE = ConfigBuilder("spark.driver.supervise") .doc("If true, restarts the driver automatically if it fails with a non-zero exit status. " + - "Only has effect in Spark standalone mode or Mesos cluster deploy mode.") + "Only has effect in Spark standalone mode.") .version("1.3.0") .booleanConf .createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index caf3c344d6462..2c4a22895a086 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -278,7 +278,4 @@ private[spark] object MetricsSystemInstances { // The Spark ApplicationMaster when running on YARN val APPLICATION_MASTER = "applicationMaster" - - // The Spark cluster scheduler when running on Mesos - val MESOS_CLUSTER = "mesos_cluster" } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 8a145223a434b..6d6197d58a10b 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -138,8 +138,8 @@ class ResourceProfile( } // Returns whether the executor cores was available to use to calculate the max tasks - // per executor and limiting resource. Some cluster managers (like standalone and coarse - // grained mesos) don't use the cores config by default so we can't use it to calculate slots. + // per executor and limiting resource. Some cluster managers (like standalone) + // don't use the cores config by default so we can't use it to calculate slots. private[spark] def isCoresLimitKnown: Boolean = _coresLimitKnown // The resource that has the least amount of slots per executor. Its possible multiple or all diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 9c8a21dccf526..81808c8a608d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -23,7 +23,7 @@ import org.apache.spark.storage.BlockManagerId /** * A backend interface for scheduling systems that allows plugging in different ones under - * TaskSchedulerImpl. We assume a Mesos-like model where the application gets resource offers as + * TaskSchedulerImpl. We assume a model where the application gets resource offers as * machines become available and can launch tasks on them. */ private[spark] trait SchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 0cb970fd27880..fefca85139838 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -1059,7 +1059,7 @@ private[spark] class TaskSchedulerImpl( case None => // We may get multiple executorLost() calls with different loss reasons. For example, // one may be triggered by a dropped connection from the worker while another may be a - // report of executor termination from Mesos. We produce log messages for both so we + // report of executor termination. We produce log messages for both so we // eventually report the termination reason. logError(s"Lost an executor $executorId (already removed): $reason") } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 69b626029e4f4..4d5f32ac67b56 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -999,7 +999,6 @@ private[spark] class TaskSetManager( } def abort(message: String, exception: Option[Throwable] = None): Unit = sched.synchronized { - // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.dagScheduler.taskSetFailed(taskSet, message, exception) isZombie = true maybeFinishTaskSet() 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 fd8e0bf3e60f7..b55dfb39d445f 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 @@ -47,9 +47,8 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils, Utils} * A scheduler backend that waits for coarse-grained executors to connect. * This backend holds onto each executor for the duration of the Spark job rather than relinquishing * executors whenever a task is done and asking the scheduler to launch a new executor for - * each new task. Executors may be launched in a variety of ways, such as Mesos tasks for the - * coarse-grained Mesos mode or standalone processes for Spark's standalone deploy mode - * (spark.deploy.*). + * each new task. Executors may be launched in a variety of ways, such as standalone processes for + * Spark's standalone deploy mode (spark.deploy.*). */ private[spark] class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: RpcEnv) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 2ec9a875d028b..48dfbecb7cda5 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -857,17 +857,7 @@ private[spark] object Utils conf.getenv("SPARK_EXECUTOR_DIRS").split(File.pathSeparator) } else if (conf.getenv("SPARK_LOCAL_DIRS") != null) { conf.getenv("SPARK_LOCAL_DIRS").split(",") - } else if (conf.getenv("MESOS_SANDBOX") != null && !shuffleServiceEnabled) { - // Mesos already creates a directory per Mesos task. Spark should use that directory - // instead so all temporary files are automatically cleaned up when the Mesos task ends. - // Note that we don't want this if the shuffle service is enabled because we want to - // continue to serve shuffle files after the executors that wrote them have already exited. - Array(conf.getenv("MESOS_SANDBOX")) } else { - if (conf.getenv("MESOS_SANDBOX") != null && shuffleServiceEnabled) { - logInfo("MESOS_SANDBOX available but not using provided Mesos sandbox because " + - s"${config.SHUFFLE_SERVICE_ENABLED.key} is enabled.") - } // In non-Yarn mode (or for the driver in yarn-client mode), we cannot trust the user // configuration to point to a secure directory. So create a subdirectory with restricted // permissions under each listed directory. @@ -992,8 +982,7 @@ private[spark] object Utils private var customHostname: Option[String] = sys.env.get("SPARK_LOCAL_HOSTNAME") /** - * Allow setting a custom host name because when we run on Mesos we need to use the same - * hostname it reports to the master. + * Allow setting a custom host name */ def setCustomHostname(hostname: String): Unit = { // DEBUG code diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index a11ecc22d0bf5..9db1afe0853a6 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -435,7 +435,7 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { } } - Seq("yarn", "local", "local[*]", "local[1,2]", "mesos://localhost:8080").foreach { master => + Seq("yarn", "local", "local[*]", "local[1,2]").foreach { master => test(s"master $master cannot use file mounted secrets") { val conf = new SparkConf() .set(AUTH_SECRET_FILE, "/tmp/secret.txt") diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 7c787f83c6a1d..f7d900b537af0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -435,28 +435,6 @@ class SparkSubmitSuite conf.get(UI_ENABLED) should be (false) } - test("handles mesos client mode") { - val clArgs = Seq( - "--deploy-mode", "client", - "--master", "mesos://h:p", - "--executor-memory", "5g", - "--total-executor-cores", "5", - "--class", "org.SomeClass", - "--driver-memory", "4g", - "--conf", "spark.ui.enabled=false", - "thejar.jar", - "arg1", "arg2") - val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, conf, mainClass) = submit.prepareSubmitEnvironment(appArgs) - childArgs.mkString(" ") should be ("arg1 arg2") - mainClass should be ("org.SomeClass") - classpath should have length (1) - classpath(0) should endWith ("thejar.jar") - conf.get("spark.executor.memory") should be ("5g") - conf.get("spark.cores.max") should be ("5") - conf.get(UI_ENABLED) should be (false) - } - test("handles k8s cluster mode") { val clArgs = Seq( "--deploy-mode", "cluster", diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 0ad14de4ab51a..3cd96670c8b5f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -407,12 +407,6 @@ class StandaloneRestSubmitSuite extends SparkFunSuite { assert(filteredVariables == Map("SPARK_VAR" -> "1")) } - test("client includes mesos env vars") { - val environmentVariables = Map("SPARK_VAR" -> "1", "MESOS_VAR" -> "1", "OTHER_VAR" -> "1") - val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables) - assert(filteredVariables == Map("SPARK_VAR" -> "1", "MESOS_VAR" -> "1")) - } - test("SPARK-45197: Make StandaloneRestServer add JavaModuleOptions to drivers") { val request = new CreateSubmissionRequest request.appResource = "" diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 59e3b69b349b8..f526eb2ff9f77 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -177,7 +177,7 @@ fi # Depending on the version being built, certain extra profiles need to be activated, and # different versions of Scala are supported. -BASE_PROFILES="-Pmesos -Pyarn -Pkubernetes" +BASE_PROFILES="-Pyarn -Pkubernetes" PUBLISH_SCALA_2_13=1 SCALA_2_13_PROFILES="-Pscala-2.13" diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 5676b362e3ee5..d7946fd7e3d9e 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -187,7 +187,6 @@ def get_commits(tag): "java api": "Java API", "k8s": "Kubernetes", "kubernetes": "Kubernetes", - "mesos": "Mesos", "ml": "MLlib", "mllib": "MLlib", "project infra": "Project Infra", diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 8c53a71c76a8f..de86d47b07359 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -181,7 +181,6 @@ log4j-core/2.20.0//log4j-core-2.20.0.jar log4j-slf4j2-impl/2.20.0//log4j-slf4j2-impl-2.20.0.jar logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.8.0//lz4-java-1.8.0.jar -mesos/1.4.3/shaded-protobuf/mesos-1.4.3-shaded-protobuf.jar metrics-core/4.2.19//metrics-core-4.2.19.jar metrics-graphite/4.2.19//metrics-graphite-4.2.19.jar metrics-jmx/4.2.19//metrics-jmx-4.2.19.jar diff --git a/dev/lint-java b/dev/lint-java index 1f0b0c8379ed0..ac5a2c869404f 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 -Pkubernetes -Pyarn -Phive -Phive-thriftserver checkstyle:check | grep ERROR) +ERRORS=$($SCRIPT_DIR/../build/mvn -Pkinesis-asl -Pkubernetes -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 859301b4d669c..17558be37fc8e 100755 --- a/dev/mima +++ b/dev/mima @@ -24,7 +24,7 @@ set -e FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" -SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"} +SPARK_PROFILES=${1:-"-Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"} TOOLS_CLASSPATH="$(build/sbt -DcopyDependencies=false "export tools/fullClasspath" | grep jar | tail -n1)" OLD_DEPS_CLASSPATH="$(build/sbt -DcopyDependencies=false $SPARK_PROFILES "export oldDeps/fullClasspath" | grep jar | tail -n1)" diff --git a/dev/sbt-checkstyle b/dev/sbt-checkstyle index 415155fd1bb19..99a46a3a0e38b 100755 --- a/dev/sbt-checkstyle +++ b/dev/sbt-checkstyle @@ -17,7 +17,7 @@ # limitations under the License. # -SPARK_PROFILES=${1:-"-Pkinesis-asl -Pmesos -Pkubernetes -Pyarn -Phive -Phive-thriftserver"} +SPARK_PROFILES=${1:-"-Pkinesis-asl -Pkubernetes -Pyarn -Phive -Phive-thriftserver"} # NOTE: echo "q" is needed because SBT prompts the user for input on encountering a build file # with failure (either resolution or compilation); the "q" makes SBT quit. diff --git a/dev/scalastyle b/dev/scalastyle index 5f958b8fb0a7b..12457af1ae7b3 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive -Pvolcano"} +SPARK_PROFILES=${1:-"-Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive -Pvolcano"} # NOTE: echo "q" is needed because SBT prompts the user for input on encountering a build file # with failure (either resolution or compilation); the "q" makes SBT quit. diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index a3bfa288383fc..8506949eaffbc 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -1158,14 +1158,6 @@ def __hash__(self): test_tags=["org.apache.spark.tags.ExtendedYarnTest"], ) -mesos = Module( - name="mesos", - dependencies=[], - source_file_regexes=["resource-managers/mesos/"], - build_profile_flags=["-Pmesos"], - sbt_test_goals=["mesos/test"], -) - kubernetes = Module( name="kubernetes", dependencies=[], diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 07da6497b97d2..855572dbc637a 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -30,7 +30,7 @@ export LC_ALL=C # NOTE: These should match those in the release publishing script, and be kept in sync with # dev/create-release/release-build.sh -HADOOP_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkubernetes -Pyarn -Phive \ +HADOOP_MODULE_PROFILES="-Phive-thriftserver -Pkubernetes -Pyarn -Phive \ -Pspark-ganglia-lgpl -Pkinesis-asl -Phadoop-cloud" MVN="build/mvn" HADOOP_HIVE_PROFILES=( diff --git a/docs/_config.yml b/docs/_config.yml index 80235fb1da57a..6455afdf22e14 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -18,12 +18,11 @@ include: - _sources # These allow the documentation to be updated with newer releases -# of Spark, Scala, and Mesos. +# of Spark, Scala. SPARK_VERSION: 4.0.0-SNAPSHOT SPARK_VERSION_SHORT: 4.0.0 SCALA_BINARY_VERSION: "2.13" SCALA_VERSION: "2.13.11" -MESOS_VERSION: 1.0.0 SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark # Before a new release, we should: diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index c2f05cfd6bb8c..03f66acb12d8e 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -105,7 +105,6 @@ Submitting Applications Spark Standalone - Mesos YARN Kubernetes diff --git a/docs/building-spark.md b/docs/building-spark.md index 63cf2deead154..95416f1e011ad 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -64,7 +64,7 @@ To create a Spark distribution like those distributed by the to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured with Maven profile settings and so on like the direct Maven build. Example: - ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes + ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phive -Phive-thriftserver -Pyarn -Pkubernetes This will build Spark distribution along with Python pip and R packages. For more information on usage, run `./dev/make-distribution.sh --help` @@ -97,10 +97,6 @@ the Spark assembly and the version on each node, included with `yarn.application The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. -## Building with Mesos support - - ./build/mvn -Pmesos -DskipTests clean package - ## Building with Kubernetes support ./build/mvn -Pkubernetes -DskipTests clean package diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 7da06a852089e..119412f96094d 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -28,7 +28,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, YARN or Kubernetes), which allocate resources across +(either Spark's own standalone cluster manager, YARN or Kubernetes), 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 @@ -48,7 +48,7 @@ There are several useful things to note about this architecture: writing it to an external storage system. 2. Spark is agnostic to the underlying cluster manager. As long as it can acquire executor processes, and these communicate with each other, it is relatively easy to run it even on a - cluster manager that also supports other applications (e.g. Mesos/YARN/Kubernetes). + cluster manager that also supports other applications (e.g. YARN/Kubernetes). 3. The driver program must listen for and accept incoming connections from its executors throughout its lifetime (e.g., see [spark.driver.port in the network config section](configuration.html#networking)). As such, the driver program must be network @@ -64,8 +64,6 @@ The system currently supports several 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. (Deprecated) * [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 3. * [Kubernetes](running-on-kubernetes.html) -- an open-source system for automating deployment, scaling, and management of containerized applications. @@ -114,7 +112,7 @@ The following table summarizes terms you'll see used to refer to cluster concept Cluster manager - An external service for acquiring resources on the cluster (e.g. standalone manager, Mesos, YARN, Kubernetes) + An external service for acquiring resources on the cluster (e.g. standalone manager, YARN, Kubernetes) Deploy mode diff --git a/docs/configuration.md b/docs/configuration.md index e9ed2a8aa37d1..39e02f36af0c7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -188,7 +188,7 @@ of the most common options to set are: Amount of non-heap memory to be allocated per driver process in cluster mode, in MiB unless otherwise specified. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size (typically 6-10%). - This option is currently supported on YARN, Mesos and Kubernetes. + This option is currently supported on YARN and Kubernetes. Note: Non-heap memory includes off-heap memory (when spark.memory.offHeap.enabled=true) and memory used by other driver processes (e.g. python process that goes with a PySpark driver) and memory used by other non-driver @@ -370,7 +370,7 @@ of the most common options to set are: comma-separated list of multiple directories on different disks.
- Note: This will be overridden by SPARK_LOCAL_DIRS (Standalone), MESOS_SANDBOX (Mesos) or + Note: This will be overridden by SPARK_LOCAL_DIRS (Standalone) or LOCAL_DIRS (YARN) environment variables set by the cluster manager. 0.5.0 @@ -426,7 +426,7 @@ of the most common options to set are: false If true, restarts the driver automatically if it fails with a non-zero exit status. - Only has effect in Spark standalone mode or Mesos cluster deploy mode. + Only has effect in Spark standalone mode. 1.3.0 @@ -2038,14 +2038,10 @@ 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 - standalone and Mesos coarse-grained modes. + 1 in YARN mode, all the available cores on the worker in standalone mode. The number of cores to use on each executor. - - In standalone and Mesos coarse-grained modes, for more detail, see - this description. 1.0.0 @@ -2057,7 +2053,6 @@ Apart from these, the following properties are also available, and may be useful with no parent RDDs, it depends on the cluster manager:
  • Local mode: number of cores on the local machine
  • -
  • Mesos fine grained mode: 8
  • Others: total number of cores on all executor nodes or 2, whichever is larger
@@ -2492,12 +2487,10 @@ Apart from these, the following properties are also available, and may be useful spark.cores.max (not set) - When running on a standalone deploy cluster or a - Mesos cluster in "coarse-grained" - sharing mode, the maximum amount of CPU cores to request for the application from + When running on a standalone deploy cluster, + the maximum amount of CPU cores to request for the application from across the cluster (not from each machine). If not set, the default will be - spark.deploy.defaultCores on Spark's standalone cluster manager, or - infinite (all available cores) on Mesos. + spark.deploy.defaultCores on Spark's standalone cluster manager. 0.6.0 @@ -2550,11 +2543,10 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.minRegisteredResourcesRatio - 0.8 for KUBERNETES mode; 0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained mode + 0.8 for KUBERNETES mode; 0.8 for YARN mode; 0.0 for standalone mode The minimum ratio of registered resources (registered resources / total expected resources) - (resources are executors in yarn mode and Kubernetes mode, CPU cores in standalone mode and Mesos coarse-grained - mode ['spark.cores.max' value is total expected resources for Mesos coarse-grained mode] ) + (resources are executors in yarn mode and Kubernetes mode, CPU cores in standalone 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, the maximum amount of time it will wait before scheduling begins is controlled by config @@ -3511,7 +3503,7 @@ External users can query the static sql config values via `SparkSession.conf` or spark.deploy.recoveryMode NONE The recovery mode setting to recover submitted Spark jobs with cluster mode when it failed and relaunches. - This is only applicable for cluster mode when running with Standalone or Mesos. + This is only applicable for cluster mode when running with Standalone. 0.8.1 @@ -3536,8 +3528,6 @@ can be found on the pages for each mode: #### [YARN](running-on-yarn.html#configuration) -#### [Mesos](running-on-mesos.html#configuration) - #### [Kubernetes](running-on-kubernetes.html#configuration) #### [Standalone Mode](spark-standalone.html#cluster-launch-scripts) @@ -3546,7 +3536,7 @@ can be found on the pages for each mode: Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh` script in the directory where Spark is installed (or `conf/spark-env.cmd` on -Windows). In Standalone and Mesos modes, this file can give machine specific information such as +Windows). In Standalone mode, this file can give machine specific information such as hostnames. It is also sourced when running local Spark applications or submission scripts. Note that `conf/spark-env.sh` does not exist by default when Spark is installed. However, you can @@ -3673,7 +3663,7 @@ There are configurations available to request resources for the driver: sp Spark will use the configurations specified to first request containers with the corresponding resources from the cluster manager. Once it gets the container, Spark launches an Executor in that container which will discover what resources the container has and the addresses associated with each resource. The Executor will register with the Driver and report back the resources available to that Executor. The Spark scheduler can then schedule tasks to each Executor and assign specific resource addresses based on the resource requirements the user specified. The user can see the resources assigned to a task using the TaskContext.get().resources api. On the driver, the user can see the resources assigned with the SparkContext resources call. It's then up to the user to use the assignedaddresses to do the processing they want or pass those into the ML/AI framework they are using. -See your cluster manager specific page for requirements and details on each of - [YARN](running-on-yarn.html#resource-allocation-and-configuration-overview), [Kubernetes](running-on-kubernetes.html#resource-allocation-and-configuration-overview) and [Standalone Mode](spark-standalone.html#resource-allocation-and-configuration-overview). It is currently not available with Mesos or local mode. And please also note that local-cluster mode with multiple workers is not supported(see Standalone documentation). +See your cluster manager specific page for requirements and details on each of - [YARN](running-on-yarn.html#resource-allocation-and-configuration-overview), [Kubernetes](running-on-kubernetes.html#resource-allocation-and-configuration-overview) and [Standalone Mode](spark-standalone.html#resource-allocation-and-configuration-overview). It is currently not available with local mode. And please also note that local-cluster mode with multiple workers is not supported(see Standalone documentation). # Stage Level Scheduling Overview diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 2464d77424059..58051d66830c6 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -26,6 +26,8 @@ license: | - Since Spark 4.0, Spark will compress event logs. To restore the behavior before Spark 4.0, you can set `spark.eventLog.compress` to `false`. +- In Spark 4.0, support for Apache Mesos as a resource manager was removed. + ## Upgrading from Core 3.3 to 3.4 - Since Spark 3.4, Spark driver will own `PersistentVolumnClaim`s and try to reuse if they are not assigned to live executors. To restore the behavior before Spark 3.4, you can set `spark.kubernetes.driver.ownPersistentVolumeClaim` to `false` and `spark.kubernetes.driver.reusePersistentVolumeClaim` to `false`. diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md index fc87995f9817f..0a008fc77036b 100644 --- a/docs/hardware-provisioning.md +++ b/docs/hardware-provisioning.md @@ -32,8 +32,7 @@ possible**. We recommend the following: Hadoop's memory and CPU usage to avoid interference (for Hadoop, the relevant options are `mapred.child.java.opts` for the per-task memory and `mapreduce.tasktracker.map.tasks.maximum` and `mapreduce.tasktracker.reduce.tasks.maximum` for number of tasks). Alternatively, you can run -Hadoop and Spark on a common cluster manager like [Mesos](running-on-mesos.html) or -[Hadoop YARN](running-on-yarn.html). +Hadoop and Spark on a common cluster manager like [Hadoop YARN](running-on-yarn.html). * If this is not possible, run Spark on different nodes in the same local-area network as HDFS. diff --git a/docs/index.md b/docs/index.md index 2735816e42e26..5f3858bec86b9 100644 --- a/docs/index.md +++ b/docs/index.md @@ -98,7 +98,6 @@ Spark can run both by itself, or over several existing cluster managers. It curr options for deployment: * [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy Spark on a private cluster -* [Apache Mesos](running-on-mesos.html) (deprecated) * [Hadoop YARN](running-on-yarn.html) * [Kubernetes](running-on-kubernetes.html) @@ -132,8 +131,6 @@ options for deployment: * Deployment modes: * [Amazon EC2](https://github.com/amplab/spark-ec2): scripts that let you launch a cluster on EC2 in about 5 minutes * [Standalone Deploy Mode](spark-standalone.html): launch a standalone cluster quickly without a third-party cluster manager - * [Mesos](running-on-mesos.html): deploy a private cluster using - [Apache Mesos](https://mesos.apache.org) * [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN) * [Kubernetes](running-on-kubernetes.html): deploy Spark on top of Kubernetes diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 0875bd5558e5b..da007608adf35 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -40,8 +40,7 @@ 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 -[coarse-grained Mesos mode](running-on-mesos.html#mesos-run-modes). +and [YARN](running-on-yarn.html) modes. Resource allocation can be configured as follows, based on the cluster type: * **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in @@ -50,23 +49,12 @@ Resource allocation can be configured as follows, based on the cluster type: or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls its memory use. -* **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 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). -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 -application is not running tasks on a machine, other applications may run tasks on those cores. This mode -is useful when you expect large numbers of not overly active applications, such as shell sessions from -separate users. However, it comes with a risk of less predictable latency, because it may take a while for -an application to gain back cores on one node when it has work to do. To use this mode, simply use a -`mesos://` URL and set `spark.mesos.coarse` to false. - Note that none of the modes currently provide memory sharing across applications. If you would like to share data this way, we recommend running a single server application that can serve multiple requests by querying the same RDDs. @@ -79,8 +67,7 @@ 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), -[Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes) and [K8s mode](running-on-kubernetes.html). +[standalone mode](spark-standalone.html), [YARN mode](running-on-yarn.html) and [K8s mode](running-on-kubernetes.html). ### Caveats @@ -103,10 +90,6 @@ without deleting shuffle files written by them (more detail described In standalone mode, simply start your workers with `spark.shuffle.service.enabled` set to `true`. -In Mesos coarse-grained mode, run `$SPARK_HOME/sbin/start-mesos-shuffle-service.sh` on all -worker nodes with `spark.shuffle.service.enabled` set to `true`. For instance, you may do so -through Marathon. - In YARN mode, follow the instructions [here](running-on-yarn.html#configuring-the-external-shuffle-service). All other relevant configurations are optional and under the `spark.dynamicAllocation.*` and @@ -201,7 +184,7 @@ mode is best for multi-user settings. 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), -[K8s mode](running-on-kubernetes.html) and [Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes). +[K8s mode](running-on-kubernetes.html). To enable the fair scheduler, simply set the `spark.scheduler.mode` property to `FAIR` when configuring a SparkContext: diff --git a/docs/monitoring.md b/docs/monitoring.md index 91b158bf85d26..a3f372bf56c41 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -1062,7 +1062,6 @@ set of sinks to which metrics are reported. The following instances are currentl * `driver`: The Spark driver process (the process in which your SparkContext is created). * `shuffleService`: The Spark shuffle service. * `applicationMaster`: The Spark ApplicationMaster when running on YARN. -* `mesos_cluster`: The Spark cluster scheduler when running on Mesos. Each instance can report to zero or more _sinks_. Sinks are contained in the `org.apache.spark.metrics.sink` package: @@ -1385,13 +1384,6 @@ Note: applies when running on YARN - numLocalityAwareTasks - numReleasedContainers -### Component instance = mesos_cluster -Note: applies when running on mesos - -- waitingDrivers -- launchedDrivers -- retryDrivers - ### Component instance = master Note: applies when running in Spark standalone as master diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index daa6588f70454..7764f0bbb5f8f 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -189,7 +189,7 @@ JavaSparkContext sc = new JavaSparkContext(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 or YARN 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-mesos.md b/docs/running-on-mesos.md deleted file mode 100644 index fef4db77f7230..0000000000000 --- a/docs/running-on-mesos.md +++ /dev/null @@ -1,901 +0,0 @@ ---- -layout: global -title: Running Spark on Mesos -license: | - 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. ---- -* This will become a table of contents (this text will be scraped). -{:toc} - -*Note*: Apache Mesos support is deprecated as of Apache Spark 3.2.0. It will be removed in a future version. - -Spark can run on hardware clusters managed by [Apache Mesos](http://mesos.apache.org/). - -The advantages of deploying Spark with Mesos include: - -- dynamic partitioning between Spark and other - [frameworks](https://mesos.apache.org/documentation/latest/frameworks/) -- scalable partitioning between multiple instances of Spark - -# Security - -Security features like authentication are not enabled by default. When deploying a cluster that is open to the internet -or an untrusted network, it's important to secure access to the cluster to prevent unauthorized applications -from running on the cluster. -Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. - -# How it Works - -In a standalone cluster deployment, the cluster manager in the below diagram is a Spark master -instance. When using Mesos, the Mesos master replaces the Spark master as the cluster manager. - -

- Spark cluster components -

- -Now when a driver creates a job and starts issuing tasks for scheduling, Mesos determines what -machines handle what tasks. Because it takes into account other frameworks when scheduling these -many short-lived tasks, multiple frameworks can coexist on the same cluster without resorting to a -static partitioning of resources. - -To get started, follow the steps below to install Mesos and deploy Spark jobs via Mesos. - - -# Installing Mesos - -Spark {{site.SPARK_VERSION}} is designed for use with Mesos {{site.MESOS_VERSION}} or newer and does not -require any special patches of Mesos. File and environment-based secrets support requires Mesos 1.3.0 or -newer. - -If you already have a Mesos cluster running, you can skip this Mesos installation step. - -Otherwise, installing Mesos for Spark is no different than installing Mesos for use by other -frameworks. You can install Mesos either from source or using prebuilt packages. - -## From Source - -To install Apache Mesos from source, follow these steps: - -1. Download a Mesos release from a - [mirror](http://www.apache.org/dyn/closer.lua/mesos/{{site.MESOS_VERSION}}/) -2. Follow the Mesos [Getting Started](http://mesos.apache.org/getting-started) page for compiling and - installing Mesos - -**Note:** If you want to run Mesos without installing it into the default paths on your system -(e.g., if you lack administrative privileges to install it), pass the -`--prefix` option to `configure` to tell it where to install. For example, pass -`--prefix=/home/me/mesos`. By default the prefix is `/usr/local`. - -## Third-Party Packages - -The Apache Mesos project only publishes source releases, not binary packages. But other -third party projects publish binary releases that may be helpful in setting Mesos up. - -One of those is Mesosphere. To install Mesos using the binary releases provided by Mesosphere: - -1. Download Mesos installation package from [downloads page](https://open.mesosphere.com/downloads/mesos/) -2. Follow their instructions for installation and configuration - -The Mesosphere installation documents suggest setting up ZooKeeper to handle Mesos master failover, -but Mesos can be run without ZooKeeper using a single master as well. - -## Verification - -To verify that the Mesos cluster is ready for Spark, navigate to the Mesos master webui at port -`:5050` Confirm that all expected machines are present in the agents tab. - - -# Connecting Spark to Mesos - -To use Mesos from Spark, you need a Spark binary package available in a place accessible by Mesos, and -a Spark driver program configured to connect to Mesos. - -Alternatively, you can also install Spark in the same location in all the Mesos agents, and configure -`spark.mesos.executor.home` (defaults to SPARK_HOME) to point to that location. - -## Authenticating to Mesos - -When Mesos Framework authentication is enabled it is necessary to provide a principal and secret by which to authenticate Spark to Mesos. Each Spark job will register with Mesos as a separate framework. - -Depending on your deployment environment you may wish to create a single set of framework credentials that are shared across all users or create framework credentials for each user. Creating and managing framework credentials should be done following the Mesos [Authentication documentation](http://mesos.apache.org/documentation/latest/authentication/). - -Framework credentials may be specified in a variety of ways depending on your deployment environment and security requirements. The most simple way is to specify the `spark.mesos.principal` and `spark.mesos.secret` values directly in your Spark configuration. Alternatively you may specify these values indirectly by instead specifying `spark.mesos.principal.file` and `spark.mesos.secret.file`, these settings point to files containing the principal and secret. These files must be plaintext files in UTF-8 encoding. Combined with appropriate file ownership and mode/ACLs this provides a more secure way to specify these credentials. - -Additionally, if you prefer to use environment variables you can specify all of the above via environment variables instead, the environment variable names are simply the configuration settings uppercased with `.` replaced with `_` e.g. `SPARK_MESOS_PRINCIPAL`. - -### Credential Specification Preference Order - -Please note that if you specify multiple ways to obtain the credentials then the following preference order applies. Spark will use the first valid value found and any subsequent values are ignored: - -- `spark.mesos.principal` configuration setting -- `SPARK_MESOS_PRINCIPAL` environment variable -- `spark.mesos.principal.file` configuration setting -- `SPARK_MESOS_PRINCIPAL_FILE` environment variable - -An equivalent order applies for the secret. Essentially we prefer the configuration to be specified directly rather than indirectly by files, and we prefer that configuration settings are used over environment variables. - -### Deploy to a Mesos running on Secure Sockets - -If you want to deploy a Spark Application into a Mesos cluster that is running in a secure mode there are some environment variables that need to be set. - -- `LIBPROCESS_SSL_ENABLED=true` enables SSL communication -- `LIBPROCESS_SSL_VERIFY_CERT=false` verifies the ssl certificate -- `LIBPROCESS_SSL_KEY_FILE=pathToKeyFile.key` path to key -- `LIBPROCESS_SSL_CERT_FILE=pathToCRTFile.crt` the certificate file to be used - -All options can be found at [http://mesos.apache.org/documentation/latest/ssl/](http://mesos.apache.org/documentation/latest/ssl/) - -Then submit happens as described in Client mode or Cluster mode below - -## Uploading Spark Package - -When Mesos runs a task on a Mesos agent for the first time, that agent must have a Spark binary -package for running the Spark Mesos executor backend. -The Spark package can be hosted at any Hadoop-accessible URI, including HTTP via `http://`, -[Amazon Simple Storage Service](http://aws.amazon.com/s3) via `s3n://`, or HDFS via `hdfs://`. - -To use a precompiled package: - -1. Download a Spark binary package from the Spark [download page](https://spark.apache.org/downloads.html) -2. Upload to hdfs/http/s3 - -To host on HDFS, use the Hadoop fs put command: `hadoop fs -put spark-{{site.SPARK_VERSION}}.tar.gz -/path/to/spark-{{site.SPARK_VERSION}}.tar.gz` - - -Or if you are using a custom-compiled version of Spark, you will need to create a package using -the `dev/make-distribution.sh` script included in a Spark source tarball/checkout. - -1. Download and build Spark using the instructions [here](index.html) -2. Create a binary package using `./dev/make-distribution.sh --tgz`. -3. Upload archive to http/s3/hdfs - - -## Using a Mesos Master URL - -The Master URLs for Mesos are in the form `mesos://host:5050` for a single-master Mesos -cluster, or `mesos://zk://host1:2181,host2:2181,host3:2181/mesos` for a multi-master Mesos cluster using ZooKeeper. - -## Client Mode - -In client mode, a Spark Mesos framework is launched directly on the client machine and waits for the driver output. - -The driver needs some configuration in `spark-env.sh` to interact properly with Mesos: - -1. In `spark-env.sh` set some environment variables: - * `export MESOS_NATIVE_JAVA_LIBRARY=`. This path is typically - `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation - instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of - `libmesos.so`. - * `export SPARK_EXECUTOR_URI=`. -2. Also set `spark.executor.uri` to ``. - -Now when starting a Spark application against the cluster, pass a `mesos://` -URL as the master when creating a `SparkContext`. For example: - -{% highlight scala %} -val conf = new SparkConf() - .setMaster("mesos://HOST:5050") - .setAppName("My app") - .set("spark.executor.uri", "") -val sc = new SparkContext(conf) -{% endhighlight %} - -(You can also use [`spark-submit`](submitting-applications.html) and configure `spark.executor.uri` -in the [conf/spark-defaults.conf](configuration.html#loading-default-configurations) file.) - -When running a shell, the `spark.executor.uri` parameter is inherited from `SPARK_EXECUTOR_URI`, so -it does not need to be redundantly passed in as a system property. - -{% highlight bash %} -./bin/spark-shell --master mesos://host:5050 -{% endhighlight %} - -## Cluster mode - -Spark on Mesos also supports cluster mode, where the driver is launched in the cluster and the client -can find the results of the driver from the Mesos Web UI. - -To use cluster mode, you must start the `MesosClusterDispatcher` in your cluster via the `sbin/start-mesos-dispatcher.sh` script, -passing in the Mesos master URL (e.g: mesos://host:5050). This starts the `MesosClusterDispatcher` as a daemon running on the host. -Note that the `MesosClusterDispatcher` does not support authentication. You should ensure that all network access to it is -protected (port 7077 by default). - -By setting the Mesos proxy config property (requires mesos version >= 1.4), `--conf spark.mesos.proxy.baseURL=http://localhost:5050` when launching the dispatcher, the mesos sandbox URI for each driver is added to the mesos dispatcher UI. - -If you like to run the `MesosClusterDispatcher` with Marathon, you need to run the `MesosClusterDispatcher` in the foreground (i.e: `./bin/spark-class org.apache.spark.deploy.mesos.MesosClusterDispatcher`). Note that the `MesosClusterDispatcher` not yet supports multiple instances for HA. - -The `MesosClusterDispatcher` also supports writing recovery state into Zookeeper. This will allow the `MesosClusterDispatcher` to be able to recover all submitted and running containers on relaunch. In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env by configuring `spark.deploy.recoveryMode` and related spark.deploy.zookeeper.* configurations. -For more information about these configurations please refer to the configurations [doc](configuration.html#deploy). - -You can also specify any additional jars required by the `MesosClusterDispatcher` in the classpath by setting the environment variable SPARK_DAEMON_CLASSPATH in spark-env. - -From the client, you can submit a job to Mesos cluster by running `spark-submit` and specifying the master URL -to the URL of the `MesosClusterDispatcher` (e.g: mesos://dispatcher:7077). You can view driver statuses on the -Spark cluster Web UI. - -For example: -{% highlight bash %} -./bin/spark-submit \ - --class org.apache.spark.examples.SparkPi \ - --master mesos://207.184.161.138:7077 \ - --deploy-mode cluster \ - --supervise \ - --executor-memory 20G \ - --total-executor-cores 100 \ - http://path/to/examples.jar \ - 1000 -{% endhighlight %} - - -Note that jars or python files that are passed to spark-submit should be URIs reachable by Mesos agents, as the Spark driver doesn't automatically upload local jars. - -# Mesos Run Modes - -Spark can run over Mesos in two modes: "coarse-grained" (default) and -"fine-grained" (deprecated). - -## Coarse-Grained - -In "coarse-grained" mode, each Spark executor runs as a single Mesos -task. Spark executors are sized according to the following -configuration variables: - -* Executor memory: `spark.executor.memory` -* Executor cores: `spark.executor.cores` -* Number of executors: `spark.cores.max`/`spark.executor.cores` - -Please see the [Spark Configuration](configuration.html) page for -details and default values. - -Executors are brought up eagerly when the application starts, until -`spark.cores.max` is reached. If you don't set `spark.cores.max`, the -Spark application will consume all resources offered to it by Mesos, -so we, of course, urge you to set this variable in any sort of -multi-tenant cluster, including one which runs multiple concurrent -Spark applications. - -The scheduler will start executors round-robin on the offers Mesos -gives it, but there are no spread guarantees, as Mesos does not -provide such guarantees on the offer stream. - -In this mode Spark executors will honor port allocation if such is -provided from the user. Specifically, if the user defines -`spark.blockManager.port` in Spark configuration, -the mesos scheduler will check the available offers for a valid port -range containing the port numbers. If no such range is available it will -not launch any task. If no restriction is imposed on port numbers by the -user, ephemeral ports are used as usual. This port honouring implementation -implies one task per host if the user defines a port. In the future network, -isolation shall be supported. - -The benefit of coarse-grained mode is much lower startup overhead, but -at the cost of reserving Mesos resources for the complete duration of -the application. To configure your job to dynamically adjust to its -resource requirements, look into -[Dynamic Allocation](#dynamic-resource-allocation-with-mesos). - -## Fine-Grained (deprecated) - -**NOTE:** Fine-grained mode is deprecated as of Spark 2.0.0. Consider - using [Dynamic Allocation](#dynamic-resource-allocation-with-mesos) - for some of the benefits. For a full explanation see - [SPARK-11857](https://issues.apache.org/jira/browse/SPARK-11857) - -In "fine-grained" mode, each Spark task inside the Spark executor runs -as a separate Mesos task. This allows multiple instances of Spark (and -other frameworks) to share cores at a very fine granularity, where -each application gets more or fewer cores as it ramps up and down, but -it comes with an additional overhead in launching each task. This mode -may be inappropriate for low-latency requirements like interactive -queries or serving web requests. - -Note that while Spark tasks in fine-grained will relinquish cores as -they terminate, they will not relinquish memory, as the JVM does not -give memory back to the Operating System. Neither will executors -terminate when they're idle. - -To run in fine-grained mode, set the `spark.mesos.coarse` property to false in your -[SparkConf](configuration.html#spark-properties): - -{% highlight scala %} -conf.set("spark.mesos.coarse", "false") -{% endhighlight %} - -You may also make use of `spark.mesos.constraints` to set -attribute-based constraints on Mesos resource offers. By default, all -resource offers will be accepted. - -{% highlight scala %} -conf.set("spark.mesos.constraints", "os:centos7;us-east-1:false") -{% endhighlight %} - -For example, Let's say `spark.mesos.constraints` is set to `os:centos7;us-east-1:false`, then the resource offers will -be checked to see if they meet both these constraints and only then will be accepted to start new executors. - -To constrain where driver tasks are run, use `spark.mesos.driver.constraints` - -# Mesos Docker Support - -Spark can make use of a Mesos Docker containerizer by setting the property `spark.mesos.executor.docker.image` -in your [SparkConf](configuration.html#spark-properties). - -The Docker image used must have an appropriate version of Spark already part of the image, or you can -have Mesos download Spark via the usual methods. - -Requires Mesos version 0.20.1 or later. - -Note that by default Mesos agents will not pull the image if it already exists on the agent. If you use mutable image -tags you can set `spark.mesos.executor.docker.forcePullImage` to `true` in order to force the agent to always pull the -image before running the executor. Force pulling images is only available in Mesos version 0.22 and above. - -# Running Alongside Hadoop - -You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a -separate service on the machines. To access Hadoop data from Spark, a full `hdfs://` URL is required -(typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode web -UI). - -In addition, it is possible to also run Hadoop MapReduce on Mesos for better resource isolation and -sharing between the two. In this case, Mesos will act as a unified scheduler that assigns cores to -either Hadoop or Spark, as opposed to having them share resources via the Linux scheduler on each -node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop). - -In either case, HDFS runs separately from Hadoop MapReduce, without being scheduled through Mesos. - -# Dynamic Resource Allocation with Mesos - -Mesos supports dynamic allocation only with coarse-grained mode, which can resize the number of -executors based on statistics of the application. For general information, -see [Dynamic Resource Allocation](job-scheduling.html#dynamic-resource-allocation). - -The External Shuffle Service to use is the Mesos Shuffle Service. It provides shuffle data cleanup functionality -on top of the Shuffle Service since Mesos doesn't yet support notifying another framework's -termination. To launch it, run `$SPARK_HOME/sbin/start-mesos-shuffle-service.sh` on all agent nodes, with `spark.shuffle.service.enabled` set to `true`. - -This can also be achieved through Marathon, using a unique host constraint, and the following command: `./bin/spark-class org.apache.spark.deploy.mesos.MesosExternalShuffleService`. - -# Configuration - -See the [configuration page](configuration.html) for information on Spark configurations. The following configs are specific for Spark on Mesos. - -#### Spark Properties - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
Property NameDefaultMeaningSince Version
spark.mesos.coarsetrue - If set to true, runs over Mesos clusters in "coarse-grained" sharing mode, where Spark acquires one long-lived Mesos task on each machine. - If set to false, runs over Mesos cluster in "fine-grained" sharing mode, where one Mesos task is created per Spark task. - Detailed information in 'Mesos Run Modes'. - 0.6.0
spark.mesos.extra.cores0 - Set the extra number of cores for an executor to advertise. This - does not result in more cores allocated. It instead means that an - executor will "pretend" it has more cores, so that the driver will - send it more tasks. Use this to increase parallelism. This - setting is only used for Mesos coarse-grained mode. - 0.6.0
spark.mesos.mesosExecutor.cores1.0 - (Fine-grained mode only) Number of cores to give each Mesos executor. This does not - include the cores used to run the Spark tasks. In other words, even if no Spark task - is being run, each Mesos executor will occupy the number of cores configured here. - The value can be a floating point number. - 1.4.0
spark.mesos.executor.docker.image(none) - Set the name of the docker image that the Spark executors will run in. The selected - image must have Spark installed, as well as a compatible version of the Mesos library. - The installed path of Spark in the image can be specified with spark.mesos.executor.home; - the installed path of the Mesos library can be specified with spark.executorEnv.MESOS_NATIVE_JAVA_LIBRARY. - 1.4.0
spark.mesos.executor.docker.forcePullImagefalse - Force Mesos agents to pull the image specified in spark.mesos.executor.docker.image. - By default Mesos agents will not pull images they already have cached. - 2.1.0
spark.mesos.executor.docker.parameters(none) - Set the list of custom parameters which will be passed into the docker run command when launching the Spark executor on Mesos using the docker containerizer. The format of this property is a comma-separated list of - key/value pairs. Example: - -
key1=val1,key2=val2,key3=val3
-
2.2.0
spark.mesos.executor.docker.volumes(none) - Set the list of volumes which will be mounted into the Docker image, which was set using - spark.mesos.executor.docker.image. The format of this property is a comma-separated list of - mappings following the form passed to docker run -v. That is they take the form: - -
[host_path:]container_path[:ro|:rw]
-
1.4.0
spark.mesos.task.labels(none) - Set the Mesos labels to add to each task. Labels are free-form key-value pairs. - Key-value pairs should be separated by a colon, and commas used to - list more than one. If your label includes a colon or comma, you - can escape it with a backslash. Ex. key:value,key2:a\:b. - 2.2.0
spark.mesos.executor.homedriver side SPARK_HOME - Set the directory in which Spark is installed on the executors in Mesos. By default, the - executors will simply use the driver's Spark home directory, which may not be visible to - them. Note that this is only relevant if a Spark binary package is not specified through - spark.executor.uri. - 1.1.1
spark.mesos.executor.memoryOverheadexecutor memory * 0.10, with minimum of 384 - The amount of additional memory, specified in MiB, to be allocated per executor. By default, - the overhead will be larger of either 384 or 10% of spark.executor.memory. If set, - the final overhead will be this value. - 1.1.1
spark.mesos.driver.memoryOverheaddriver memory * 0.10, with minimum of 384 - The amount of additional memory, specified in MB, to be allocated to the driver. By default, - the overhead will be larger of either 384 or 10% of spark.driver.memory. If set, - the final overhead will be this value. Only applies to cluster mode. -
spark.mesos.uris(none) - A comma-separated list of URIs to be downloaded to the sandbox - when driver or executor is launched by Mesos. This applies to - both coarse-grained and fine-grained mode. - 1.5.0
spark.mesos.principal(none) - Set the principal with which Spark framework will use to authenticate with Mesos. You can also specify this via the environment variable `SPARK_MESOS_PRINCIPAL`. - 1.5.0
spark.mesos.principal.file(none) - Set the file containing the principal with which Spark framework will use to authenticate with Mesos. Allows specifying the principal indirectly in more security conscious deployments. The file must be readable by the user launching the job and be UTF-8 encoded plaintext. You can also specify this via the environment variable `SPARK_MESOS_PRINCIPAL_FILE`. - 2.4.0
spark.mesos.secret(none) - Set the secret with which Spark framework will use to authenticate with Mesos. Used, for example, when - authenticating with the registry. You can also specify this via the environment variable `SPARK_MESOS_SECRET`. - 1.5.0
spark.mesos.secret.file(none) - Set the file containing the secret with which Spark framework will use to authenticate with Mesos. Used, for example, when - authenticating with the registry. Allows for specifying the secret indirectly in more security conscious deployments. The file must be readable by the user launching the job and be UTF-8 encoded plaintext. You can also specify this via the environment variable `SPARK_MESOS_SECRET_FILE`. - 2.4.0
spark.mesos.role* - Set the role of this Spark framework for Mesos. Roles are used in Mesos for reservations - and resource weight sharing. - 1.5.0
spark.mesos.constraints(none) - Attribute-based constraints on mesos resource offers. By default, all resource offers will be accepted. This setting - applies only to executors. Refer to Mesos - Attributes & Resources for more information on attributes. -
    -
  • Scalar constraints are matched with "less than equal" semantics i.e. value in the constraint must be less than or equal to the value in the resource offer.
  • -
  • Range constraints are matched with "contains" semantics i.e. value in the constraint must be within the resource offer's value.
  • -
  • Set constraints are matched with "subset of" semantics i.e. value in the constraint must be a subset of the resource offer's value.
  • -
  • Text constraints are matched with "equality" semantics i.e. value in the constraint must be exactly equal to the resource offer's value.
  • -
  • In case there is no value present as a part of the constraint any offer with the corresponding attribute will be accepted (without value check).
  • -
-
1.5.0
spark.mesos.driver.constraints(none) - Same as spark.mesos.constraints except applied to drivers when launched through the dispatcher. By default, - all offers with sufficient resources will be accepted. - 2.2.1
spark.mesos.containerizerdocker - This only affects docker containers, and must be one of "docker" - or "mesos". Mesos supports two types of - containerizers for docker: the "docker" containerizer, and the preferred - "mesos" containerizer. Read more here. - 2.1.0
spark.mesos.driver.webui.url(none) - Set the Spark Mesos driver webui_url for interacting with the framework. - If unset it will point to Spark's internal web UI. - 2.0.0
spark.mesos.driver.labels(none) - Mesos labels to add to the driver. See spark.mesos.task.labels - for formatting information. - 2.3.0
- spark.mesos.driver.secret.values, - spark.mesos.driver.secret.names, - spark.mesos.executor.secret.values, - spark.mesos.executor.secret.names, - (none) -

- A secret is specified by its contents and destination. These properties - specify a secret's contents. To specify a secret's destination, see the cell below. -

-

- You can specify a secret's contents either (1) by value or (2) by reference. -

-

- (1) To specify a secret by value, set the - spark.mesos.[driver|executor].secret.values - property, to make the secret available in the driver or executors. - For example, to make a secret password "guessme" available to the driver process, set: - -

spark.mesos.driver.secret.values=guessme
-

-

- (2) To specify a secret that has been placed in a secret store - by reference, specify its name within the secret store - by setting the spark.mesos.[driver|executor].secret.names - property. For example, to make a secret password named "password" in a secret store - available to the driver process, set: - -

spark.mesos.driver.secret.names=password
-

-

- Note: To use a secret store, make sure one has been integrated with Mesos via a custom - SecretResolver - module. -

-

- To specify multiple secrets, provide a comma-separated list: - -

spark.mesos.driver.secret.values=guessme,passwd123
- - or - -
spark.mesos.driver.secret.names=password1,password2
-

-
2.3.0
- spark.mesos.driver.secret.envkeys, - spark.mesos.driver.secret.filenames, - spark.mesos.executor.secret.envkeys, - spark.mesos.executor.secret.filenames, - (none) -

- A secret is specified by its contents and destination. These properties - specify a secret's destination. To specify a secret's contents, see the cell above. -

-

- You can specify a secret's destination in the driver or - executors as either (1) an environment variable or (2) as a file. -

-

- (1) To make an environment-based secret, set the - spark.mesos.[driver|executor].secret.envkeys property. - The secret will appear as an environment variable with the - given name in the driver or executors. For example, to make a secret password available - to the driver process as $PASSWORD, set: - -

spark.mesos.driver.secret.envkeys=PASSWORD
-

-

- (2) To make a file-based secret, set the - spark.mesos.[driver|executor].secret.filenames property. - The secret will appear in the contents of a file with the given file name in - the driver or executors. For example, to make a secret password available in a - file named "pwdfile" in the driver process, set: - -

spark.mesos.driver.secret.filenames=pwdfile
-

-

- Paths are relative to the container's work directory. Absolute paths must - already exist. Note: File-based secrets require a custom - SecretResolver - module. -

-

- To specify env vars or file names corresponding to multiple secrets, - provide a comma-separated list: - -

spark.mesos.driver.secret.envkeys=PASSWORD1,PASSWORD2
- - or - -
spark.mesos.driver.secret.filenames=pwdfile1,pwdfile2
-

-
2.3.0
spark.mesos.driverEnv.[EnvironmentVariableName](none) - This only affects drivers submitted in cluster mode. Add the - environment variable specified by EnvironmentVariableName to the - driver process. The user can specify multiple of these to set - multiple environment variables. - 2.1.0
spark.mesos.dispatcher.webui.url(none) - Set the Spark Mesos dispatcher webui_url for interacting with the framework. - If unset it will point to Spark's internal web UI. - 2.0.0
spark.mesos.dispatcher.driverDefault.[PropertyName](none) - Set default properties for drivers submitted through the - dispatcher. For example, - spark.mesos.dispatcher.driverProperty.spark.executor.memory=32g - results in the executors for all drivers submitted in cluster mode - to run in 32g containers. - 2.1.0
spark.mesos.dispatcher.historyServer.url(none) - Set the URL of the history - server. The dispatcher will then link each driver to its entry - in the history server. - 2.1.0
spark.mesos.dispatcher.queue(none) - Set the name of the dispatcher queue to which the application is submitted. - The specified queue must be added to the dispatcher with spark.mesos.dispatcher.queue.[QueueName]. - If no queue is specified, then the application is submitted to the "default" queue with 0.0 priority. - 3.1.0
spark.mesos.dispatcher.queue.[QueueName]0.0 - Add a new queue for submitted drivers with the specified priority. - Higher numbers indicate higher priority. - The user can specify multiple queues to define a workload management policy for queued drivers in the dispatcher. - A driver can then be submitted to a specific queue with spark.mesos.dispatcher.queue. - By default, the dispatcher has a single queue with 0.0 priority (cannot be overridden). - It is possible to implement a consistent and overall workload management policy throughout the lifecycle of drivers - by mapping priority queues to weighted Mesos roles, and by specifying a - spark.mesos.role along with a spark.mesos.dispatcher.queue when submitting an application. - For example, with the URGENT Mesos role: -
-    spark.mesos.dispatcher.queue.URGENT=1.0
-
-    spark.mesos.dispatcher.queue=URGENT
-    spark.mesos.role=URGENT
-    
-
3.1.0
spark.mesos.gpus.max0 - Set the maximum number GPU resources to acquire for this job. Note that executors will still launch when no GPU resources are found - since this configuration is just an upper limit and not a guaranteed amount. - 2.1.0
spark.mesos.network.name(none) - Attach containers to the given named network. If this job is - launched in cluster mode, also launch the driver in the given named - network. See - the Mesos CNI docs - for more details. - 2.1.0
spark.mesos.network.labels(none) - Pass network labels to CNI plugins. This is a comma-separated list - of key-value pairs, where each key-value pair has the format key:value. - Example: - -
key1:val1,key2:val2
- See - the Mesos CNI docs - for more details. -
2.3.0
spark.mesos.fetcherCache.enablefalse - If set to `true`, all URIs (example: `spark.executor.uri`, - `spark.mesos.uris`) will be cached by the Mesos - Fetcher Cache - 2.1.0
spark.mesos.driver.failoverTimeout0.0 - The amount of time (in seconds) that the master will wait for the - driver to reconnect, after being temporarily disconnected, before - it tears down the driver framework by killing all its - executors. The default value is zero, meaning no timeout: if the - driver disconnects, the master immediately tears down the framework. - 2.3.0
spark.mesos.rejectOfferDuration120s - Time to consider unused resources refused, serves as a fallback of - `spark.mesos.rejectOfferDurationForUnmetConstraints`, - `spark.mesos.rejectOfferDurationForReachedMaxCores` - 2.2.0
spark.mesos.rejectOfferDurationForUnmetConstraintsspark.mesos.rejectOfferDuration - Time to consider unused resources refused with unmet constraints - 1.6.0
spark.mesos.rejectOfferDurationForReachedMaxCoresspark.mesos.rejectOfferDuration - Time to consider unused resources refused when maximum number of cores - spark.cores.max is reached - 2.0.0
spark.mesos.appJar.local.resolution.modehost - Provides support for the `local:///` scheme to reference the app jar resource in cluster mode. - If user uses a local resource (`local:///path/to/jar`) and the config option is not used it defaults to `host` e.g. - the mesos fetcher tries to get the resource from the host's file system. - If the value is unknown it prints a warning msg in the dispatcher logs and defaults to `host`. - If the value is `container` then spark submit in the container will use the jar in the container's path: - `/path/to/jar`. - 2.4.0
- -# Troubleshooting and Debugging - -A few places to look during debugging: - -- Mesos master on port `:5050` - - Agents should appear in the agents tab - - Spark applications should appear in the frameworks tab - - Tasks should appear in the details of a framework - - Check the stdout and stderr of the sandbox of failed tasks -- Mesos logs - - Master and agent logs are both in `/var/log/mesos` by default - -And common pitfalls: - -- Spark assembly not reachable/accessible - - Agents must be able to download the Spark binary package from the `http://`, `hdfs://` or `s3n://` URL you gave -- Firewall blocking communications - - Check for messages about failed connections - - Temporarily disable firewalls for debugging and then poke appropriate holes diff --git a/docs/security.md b/docs/security.md index 30cfd660c41bf..0a6a3cf089185 100644 --- a/docs/security.md +++ b/docs/security.md @@ -49,9 +49,9 @@ specified below, the secret must be defined by setting the `spark.authenticate.s option. The same secret is shared by all Spark applications and daemons in that case, which limits the security of these deployments, especially on multi-tenant clusters. -The REST Submission Server and the MesosClusterDispatcher do not support authentication. You should -ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077 -respectively by default) are restricted to hosts that are trusted to submit jobs. +The REST Submission Server does not support authentication. You should +ensure that all network access to the REST API (port 6066 by default) +is restricted to hosts that are trusted to submit jobs. ### YARN @@ -225,7 +225,7 @@ The following settings cover enabling encryption for data written to disk: spark.io.encryption.enabled false - Enable local disk I/O encryption. Currently supported by all modes except Mesos. It's strongly + Enable local disk I/O encryption. Currently supported by all modes. It's strongly recommended that RPC encryption be enabled when using this feature. 2.1.0 @@ -621,20 +621,6 @@ The user may allow the executors to use the SSL settings inherited from the work can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. In that case, the settings provided by the user on the client side are not used. -### Mesos mode - -Mesos 1.3.0 and newer supports `Secrets` primitives as both file-based and environment based -secrets. Spark allows the specification of file-based and environment variable based secrets with -`spark.mesos.driver.secret.filenames` and `spark.mesos.driver.secret.envkeys`, respectively. - -Depending on the secret store backend secrets can be passed by reference or by value with the -`spark.mesos.driver.secret.names` and `spark.mesos.driver.secret.values` configuration properties, -respectively. - -Reference type secrets are served by the secret store and referred to by name, for example -`/mysecret`. Value type secrets are passed on the command line and translated into their -appropriate files or environment variables. - ## HTTP Security Headers Apache Spark can be configured to include HTTP headers to aid in preventing Cross Site Scripting @@ -819,7 +805,7 @@ mechanism (see `java.util.ServiceLoader`). Implementations of `org.apache.spark.security.HadoopDelegationTokenProvider` can be made available to Spark by listing their names in the corresponding file in the jar's `META-INF/services` directory. -Delegation token support is currently only supported in YARN and Mesos modes. Consult the +Delegation token support is currently only supported in YARN mode. Consult the deployment-specific page for more information. The following options provides finer-grained control for this feature: @@ -861,7 +847,7 @@ Long-running applications may run into issues if their run time exceeds the maxi token lifetime configured in services it needs to access. This feature is not available everywhere. In particular, it's only implemented -on YARN and Kubernetes (both client and cluster modes), and on Mesos when using client mode. +on YARN and Kubernetes (both client and cluster modes). Spark supports automatically creating new tokens for these applications. There are two ways to enable this functionality. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 3e87edad0aadd..b60ad3e80a4af 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -21,7 +21,7 @@ license: | * This will become a table of contents (this text will be scraped). {:toc} -In addition to running on the Mesos or YARN cluster managers, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided [launch scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing. +In addition to running on the YARN cluster manager, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided [launch scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing. # Security diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index c2ad50d2a626b..7814de818d4cf 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -465,7 +465,7 @@ ssc = StreamingContext(sc, 1) {% endhighlight %} 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 or YARN 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 @@ -490,7 +490,7 @@ val ssc = new StreamingContext(conf, Seconds(1)) {% endhighlight %} The `appName` parameter is a name for your application to show on the cluster UI. -`master` is a [Spark, Mesos, Kubernetes or YARN cluster URL](submitting-applications.html#master-urls), +`master` is a [Spark, Kubernetes or YARN 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 @@ -526,7 +526,7 @@ JavaStreamingContext ssc = new JavaStreamingContext(conf, new Duration(1000)); {% endhighlight %} 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 or YARN 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 @@ -2093,8 +2093,6 @@ To run Spark Streaming applications, you need to have the following. [Spark Standalone guide](spark-standalone.html) for more details. + *YARN* - Yarn supports a similar mechanism for automatically restarting an application. Please refer to YARN documentation for more details. - + *Mesos* - [Marathon](https://github.com/mesosphere/marathon) has been used to achieve this - with Mesos. - *Configuring write-ahead logs* - Since Spark 1.2, we have introduced _write-ahead logs_ for achieving strong @@ -2291,19 +2289,6 @@ In both cases, using Kryo serialization can reduce both CPU and memory overheads In specific cases where the amount of data that needs to be retained for the streaming application is not large, it may be feasible to persist data (both types) as deserialized objects without incurring excessive GC overheads. For example, if you are using batch intervals of a few seconds and no window operations, then you can try disabling serialization in persisted data by explicitly setting the storage level accordingly. This would reduce the CPU overheads due to serialization, potentially improving performance without too much GC overheads. -### Task Launching Overheads -{:.no_toc} -If the number of tasks launched per second is high (say, 50 or more per second), then the overhead -of sending out tasks to the executors may be significant and will make it hard to achieve sub-second -latencies. The overhead can be reduced by the following changes: - -* **Execution mode**: Running Spark in Standalone mode or coarse-grained Mesos mode leads to - better task launch times than the fine-grained Mesos mode. Please refer to the - [Running on Mesos guide](running-on-mesos.html) for more details. - -These changes may reduce batch processing time by 100s of milliseconds, -thus allowing sub-second batch size to be viable. - *** ## Setting the Right Batch Interval diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index becdfb4b18f5d..30da917339cc9 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -132,17 +132,6 @@ export HADOOP_CONF_DIR=XXX examples/src/main/python/pi.py \ 1000 -# Run on a Mesos cluster in cluster deploy mode with supervise -./bin/spark-submit \ - --class org.apache.spark.examples.SparkPi \ - --master mesos://207.184.161.138:7077 \ - --deploy-mode cluster \ - --supervise \ - --executor-memory 20G \ - --total-executor-cores 100 \ - http://path/to/examples.jar \ - 1000 - # Run on a Kubernetes cluster in cluster deploy mode ./bin/spark-submit \ --class org.apache.spark.examples.SparkPi \ @@ -173,11 +162,6 @@ The master URL passed to Spark can be in one of the following formats: spark://HOST1:PORT1,HOST2:PORT2 Connect to the given Spark standalone cluster with standby masters with Zookeeper. The list must have all the master hosts in the high availability cluster set up with Zookeeper. The port must be whichever each master is configured to use, which is 7077 by default. - mesos://HOST:PORT Connect to the given Mesos cluster. - The port must be whichever one your is configured to use, which is 5050 by default. - Or, for a Mesos cluster using ZooKeeper, use mesos://zk://.... - To submit with --deploy-mode cluster, the HOST:PORT should be configured to connect to the MesosClusterDispatcher. - yarn Connect to a YARN cluster in client or cluster mode depending on the value of --deploy-mode. The cluster location will be found based on the HADOOP_CONF_DIR or YARN_CONF_DIR variable. 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 2bbeac25c0693..21861bdcb55e5 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -156,7 +156,6 @@ List buildClassPath(String appClassPath) throws IOException { "launcher", "mllib", "repl", - "resource-managers/mesos", "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 a9daf0e25722a..4ea9cd5a44d25 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java @@ -47,7 +47,7 @@ public List buildCommand(Map env) String memKey = null; String extraClassPath = null; - // Master, Worker, HistoryServer, ExternalShuffleService, MesosClusterDispatcher use + // Master, Worker, HistoryServer, ExternalShuffleService use // SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. switch (className) { case "org.apache.spark.deploy.master.Master": @@ -73,18 +73,7 @@ public List buildCommand(Map env) memKey = "SPARK_EXECUTOR_MEMORY"; extraClassPath = getenv("SPARK_EXECUTOR_CLASSPATH"); break; - case "org.apache.spark.executor.MesosExecutorBackend": - 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"); - extraClassPath = getenv("SPARK_DAEMON_CLASSPATH"); - memKey = "SPARK_DAEMON_MEMORY"; - break; case "org.apache.spark.deploy.ExternalShuffleService": - case "org.apache.spark.deploy.mesos.MesosExternalShuffleService": javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); javaOptsKeys.add("SPARK_SHUFFLE_OPTS"); extraClassPath = getenv("SPARK_DAEMON_CLASSPATH"); diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index f7d02646198e8..31318f1318ca0 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -305,10 +305,6 @@ public void testIsClientMode() { Map userProps = new HashMap<>(); userProps.put("spark.submit.deployMode", "client"); assertTrue(builder.isClientMode(userProps)); - // --master mesos --deploy-mode cluster - sparkSubmitArgs = Arrays.asList(parser.MASTER, "mesos", parser.DEPLOY_MODE, "cluster"); - builder = newCommandBuilder(sparkSubmitArgs); - assertFalse(builder.isClientMode(Collections.emptyMap())); } private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) throws Exception { diff --git a/pom.xml b/pom.xml index 20ca97dc49893..8a34f48382fc6 100644 --- a/pom.xml +++ b/pom.xml @@ -3598,13 +3598,6 @@
- - mesos - - resource-managers/mesos - - - kubernetes diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 66a8b5a6f7fad..85ffda304bc1e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -64,10 +64,10 @@ object BuildCommons { "tags", "sketch", "kvstore", "common-utils", "sql-api" ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects ++ Seq(connectCommon, connect, connectClient) - val optionallyEnabledProjects@Seq(kubernetes, mesos, yarn, + val optionallyEnabledProjects@Seq(kubernetes, yarn, sparkGangliaLgpl, streamingKinesisAsl, dockerIntegrationTests, hadoopCloud, kubernetesIntegrationTests) = - Seq("kubernetes", "mesos", "yarn", + Seq("kubernetes", "yarn", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests", "hadoop-cloud", "kubernetes-integration-tests").map(ProjectRef(buildLocation, _)) diff --git a/python/README.md b/python/README.md index 088054e3bb42f..1162990a18f7b 100644 --- a/python/README.md +++ b/python/README.md @@ -22,7 +22,7 @@ This packaging is currently experimental and may change in future versions (alth Using PySpark requires the Spark JARs, and if you are building this from source please see the builder instructions at ["Building Spark"](https://spark.apache.org/docs/latest/building-spark.html). -The Python packaging for Spark is not intended to replace all of the other use cases. This Python packaged version of Spark is suitable for interacting with an existing cluster (be it Spark standalone, YARN, or Mesos) - but does not contain the tools required to set up your own standalone Spark cluster. You can download the full version of Spark from the [Apache Spark downloads page](https://spark.apache.org/downloads.html). +The Python packaging for Spark is not intended to replace all of the other use cases. This Python packaged version of Spark is suitable for interacting with an existing cluster (be it Spark standalone, YARN) - but does not contain the tools required to set up your own standalone Spark cluster. You can download the full version of Spark from the [Apache Spark downloads page](https://spark.apache.org/downloads.html). **NOTE:** If you are using this with a Spark standalone cluster you must ensure that the version (including minor version) matches or you may experience odd errors. diff --git a/python/docs/source/user_guide/python_packaging.rst b/python/docs/source/user_guide/python_packaging.rst index 0284313508999..3adfb5b4b371b 100644 --- a/python/docs/source/user_guide/python_packaging.rst +++ b/python/docs/source/user_guide/python_packaging.rst @@ -20,7 +20,7 @@ Python Package Management ========================= -When you want to run your PySpark application on a cluster such as YARN, Kubernetes, Mesos, etc., you need to make +When you want to run your PySpark application on a cluster such as YARN, Kubernetes, etc., you need to make sure that your code and all used libraries are available on the executors. As an example, let's say you may want to run the `Pandas UDF examples `_. diff --git a/python/pyspark/context.py b/python/pyspark/context.py index b557970a07d82..e4c8b2378721c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -102,7 +102,7 @@ class SparkContext: Parameters ---------- master : str, optional - Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + Cluster URL to connect to (e.g. spark://host:port, local[4]). appName : str, optional A name for your job, to display on the cluster web UI. sparkHome : str, optional diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index b9f44a707465e..1a7be083d2d92 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -148,30 +148,6 @@ class ReplSuite extends SparkFunSuite { assertContains("res2: Array[Int] = Array(5, 0, 0, 0, 0)", output) } - if (System.getenv("MESOS_NATIVE_JAVA_LIBRARY") != null) { - test("running on Mesos") { - val output = runInterpreter("localquiet", - """ - |var v = 7 - |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) - |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) - |var array = new Array[Int](5) - |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() - |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() - """.stripMargin) - assertDoesNotContain("error:", output) - assertDoesNotContain("Exception", output) - assertContains("res0: Int = 70", output) - assertContains("res1: Int = 100", output) - assertContains("res2: Array[Int] = Array(0, 0, 0, 0, 0)", output) - assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output) - } - } - test("line wrapper only initialized once when used as encoder outer scope") { val output = runInterpreter("local", """ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala index 6396b99d5d7f2..648d2fa42eb33 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala @@ -45,7 +45,7 @@ private[spark] class LocalDirsFeatureStep( // property - we want to instead default to mounting an emptydir volume that doesn't already // exist in the image. // We could make utils.getConfiguredLocalDirs opinionated about Kubernetes, as it is already - // a bit opinionated about YARN and Mesos. + // a bit opinionated about YARN. val resolvedLocalDirs = Option(conf.sparkConf.getenv("SPARK_LOCAL_DIRS")) .orElse(conf.getOption("spark.local.dir")) .getOrElse(defaultLocalDir) diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml deleted file mode 100644 index da0f31996350b..0000000000000 --- a/resource-managers/mesos/pom.xml +++ /dev/null @@ -1,128 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - spark-parent_2.13 - 4.0.0-SNAPSHOT - ../../pom.xml - - - spark-mesos_2.13 - jar - Spark Project Mesos - - mesos - 1.4.3 - shaded-protobuf - - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - - org.apache.spark - spark-tags_${scala.binary.version} - test-jar - test - - - - org.apache.mesos - mesos - ${mesos.version} - ${mesos.classifier} - - - com.google.protobuf - protobuf-java - - - - - - org.mockito - mockito-core - test - - - - ${hive.group} - hive-exec - ${hive.classifier} - provided - - - ${hive.group} - hive-metastore - provided - - - - - com.google.guava - guava - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-plus - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - jetty-http - - - org.eclipse.jetty - jetty-servlet - - - org.eclipse.jetty - jetty-servlets - - - - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - diff --git a/resource-managers/mesos/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalBlockStoreClient.java b/resource-managers/mesos/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalBlockStoreClient.java deleted file mode 100644 index 3f7d31eff3165..0000000000000 --- a/resource-managers/mesos/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalBlockStoreClient.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.shuffle.mesos; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.spark.network.shuffle.protocol.mesos.ShuffleServiceHeartbeat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.sasl.SecretKeyHolder; -import org.apache.spark.network.shuffle.ExternalBlockStoreClient; -import org.apache.spark.network.shuffle.protocol.mesos.RegisterDriver; -import org.apache.spark.network.util.TransportConf; - -/** - * A client for talking to the external shuffle service in Mesos coarse-grained mode. - * - * This is used by the Spark driver to register with each external shuffle service on the cluster. - * The reason why the driver has to talk to the service is for cleaning up shuffle files reliably - * after the application exits. Mesos does not provide a great alternative to do this, so Spark - * has to detect this itself. - */ -public class MesosExternalBlockStoreClient extends ExternalBlockStoreClient { - private static final Logger logger = - LoggerFactory.getLogger(MesosExternalBlockStoreClient.class); - - private final ScheduledExecutorService heartbeaterThread = - Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("mesos-external-shuffle-client-heartbeater") - .build()); - - /** - * Creates an Mesos external shuffle client that wraps the {@link ExternalBlockStoreClient}. - * Please refer to docs on {@link ExternalBlockStoreClient} for more information. - */ - public MesosExternalBlockStoreClient( - TransportConf conf, - SecretKeyHolder secretKeyHolder, - boolean authEnabled, - long registrationTimeoutMs) { - super(conf, secretKeyHolder, authEnabled, registrationTimeoutMs); - } - - public void registerDriverWithShuffleService( - String host, - int port, - long heartbeatTimeoutMs, - long heartbeatIntervalMs) throws IOException, InterruptedException { - - checkInit(); - ByteBuffer registerDriver = new RegisterDriver(appId, heartbeatTimeoutMs).toByteBuffer(); - TransportClient client = clientFactory.createClient(host, port); - client.sendRpc(registerDriver, new RegisterDriverCallback(client, heartbeatIntervalMs)); - } - - private class RegisterDriverCallback implements RpcResponseCallback { - private final TransportClient client; - private final long heartbeatIntervalMs; - - private RegisterDriverCallback(TransportClient client, long heartbeatIntervalMs) { - this.client = client; - this.heartbeatIntervalMs = heartbeatIntervalMs; - } - - @Override - public void onSuccess(ByteBuffer response) { - heartbeaterThread.scheduleAtFixedRate( - new Heartbeater(client), 0, heartbeatIntervalMs, TimeUnit.MILLISECONDS); - logger.info("Successfully registered app " + appId + " with external shuffle service."); - } - - @Override - public void onFailure(Throwable e) { - logger.warn("Unable to register app " + appId + " with external shuffle service. " + - "Please manually remove shuffle data after driver exit. Error: " + e); - } - } - - @Override - public void close() { - heartbeaterThread.shutdownNow(); - super.close(); - } - - private class Heartbeater implements Runnable { - - private final TransportClient client; - - private Heartbeater(TransportClient client) { - this.client = client; - } - - @Override - public void run() { - // TODO: Stop sending heartbeats if the shuffle service has lost the app due to timeout - client.send(new ShuffleServiceHeartbeat(appId).toByteBuffer()); - } - } -} diff --git a/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager deleted file mode 100644 index f83bfa166bec8..0000000000000 --- a/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager +++ /dev/null @@ -1,18 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -org.apache.spark.scheduler.cluster.mesos.MesosClusterManager diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala deleted file mode 100644 index c6139dd268d5c..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ /dev/null @@ -1,136 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import java.util.Locale -import java.util.concurrent.CountDownLatch - -import org.apache.spark.{SecurityManager, SparkConf, SparkException} -import org.apache.spark.deploy.mesos.config._ -import org.apache.spark.deploy.mesos.ui.MesosClusterUI -import org.apache.spark.deploy.rest.mesos.MesosRestServer -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.Deploy._ -import org.apache.spark.scheduler.cluster.mesos._ -import org.apache.spark.util.{CommandLineUtils, ShutdownHookManager, SparkUncaughtExceptionHandler, Utils} - -/* - * A dispatcher that is responsible for managing and launching drivers, and is intended to be - * used for Mesos cluster mode. The dispatcher is a long-running process started by the user in - * the cluster independently of Spark applications. - * It contains a [[MesosRestServer]] that listens for requests to submit drivers and a - * [[MesosClusterScheduler]] that processes these requests by negotiating with the Mesos master - * for resources. - * - * A typical new driver lifecycle is the following: - * - Driver submitted via spark-submit talking to the [[MesosRestServer]] - * - [[MesosRestServer]] queues the driver request to [[MesosClusterScheduler]] - * - [[MesosClusterScheduler]] gets resource offers and launches the drivers that are in queue - * - * This dispatcher supports both Mesos fine-grain or coarse-grain mode as the mode is configurable - * per driver launched. - * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as - * a daemon to launch drivers as Mesos frameworks upon request. The dispatcher is also started and - * stopped by sbin/start-mesos-dispatcher and sbin/stop-mesos-dispatcher respectively. - */ -private[mesos] class MesosClusterDispatcher( - args: MesosClusterDispatcherArguments, - conf: SparkConf) - extends Logging { - - { - // This doesn't support authentication because the RestSubmissionServer doesn't support it. - val authKey = SecurityManager.SPARK_AUTH_SECRET_CONF - require(conf.getOption(authKey).isEmpty, - s"The MesosClusterDispatcher does not support authentication via ${authKey}. It is not " + - s"currently possible to run jobs in cluster mode with authentication on.") - } - - private val publicAddress = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(args.host) - private val recoveryMode = conf.get(RECOVERY_MODE).toUpperCase(Locale.ROOT) - logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode) - - private val engineFactory = recoveryMode match { - case "NONE" => new BlackHoleMesosClusterPersistenceEngineFactory - case "ZOOKEEPER" => new ZookeeperMesosClusterPersistenceEngineFactory(conf) - case _ => throw new IllegalArgumentException("Unsupported recovery mode: " + recoveryMode) - } - - private val scheduler = new MesosClusterScheduler(engineFactory, conf) - - private val server = new MesosRestServer(args.host, args.port, conf, scheduler) - private val webUi = new MesosClusterUI( - new SecurityManager(conf), - args.webUiPort, - conf, - publicAddress, - scheduler) - - private val shutdownLatch = new CountDownLatch(1) - - def start(): Unit = { - webUi.bind() - scheduler.frameworkUrl = conf.get(DISPATCHER_WEBUI_URL).getOrElse(webUi.activeWebUiUrl) - scheduler.start() - server.start() - } - - def awaitShutdown(): Unit = { - shutdownLatch.await() - } - - def stop(): Unit = { - webUi.stop() - server.stop() - scheduler.stop() - shutdownLatch.countDown() - } -} - -private[mesos] object MesosClusterDispatcher - extends Logging - with CommandLineUtils { - - override def main(args: Array[String]): Unit = { - Thread.setDefaultUncaughtExceptionHandler(new SparkUncaughtExceptionHandler) - Utils.initDaemon(log) - val conf = new SparkConf - val dispatcherArgs = try { - new MesosClusterDispatcherArguments(args, conf) - } catch { - case e: SparkException => - printErrorAndExit(e.getMessage()) - null - } - conf.setMaster(dispatcherArgs.masterUrl) - conf.setAppName(dispatcherArgs.name) - dispatcherArgs.zookeeperUrl.foreach { z => - conf.set(RECOVERY_MODE, "ZOOKEEPER") - conf.set(ZOOKEEPER_URL, z) - } - val dispatcher = new MesosClusterDispatcher(dispatcherArgs, conf) - dispatcher.start() - logDebug("Adding shutdown hook") // force eager creation of logger - ShutdownHookManager.addShutdownHook { () => - logInfo("Shutdown hook is shutting down dispatcher") - dispatcher.stop() - dispatcher.awaitShutdown() - } - dispatcher.awaitShutdown() - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala deleted file mode 100644 index 267a4283db9e6..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import scala.annotation.tailrec -import scala.collection.mutable - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkSubmitUtils -import org.apache.spark.util.{IntParam, Utils} - -private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: SparkConf) { - var host: String = Utils.localHostName() - var port: Int = 7077 - var name: String = "Spark Cluster" - var webUiPort: Int = 8081 - var verbose: Boolean = false - var masterUrl: String = _ - var zookeeperUrl: Option[String] = None - var propertiesFile: String = _ - val confProperties: mutable.HashMap[String, String] = - new mutable.HashMap[String, String]() - - parse(args.toList) - - // scalastyle:on println - propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - Utils.updateSparkConfigFromProperties(conf, confProperties) - - // scalastyle:off println - if (verbose) { - MesosClusterDispatcher.printStream.println(s"Using host: $host") - MesosClusterDispatcher.printStream.println(s"Using port: $port") - MesosClusterDispatcher.printStream.println(s"Using webUiPort: $webUiPort") - MesosClusterDispatcher.printStream.println(s"Framework Name: $name") - - Option(propertiesFile).foreach { file => - MesosClusterDispatcher.printStream.println(s"Using properties file: $file") - } - - MesosClusterDispatcher.printStream.println(s"Spark Config properties set:") - conf.getAll.foreach(println) - } - - @tailrec - private def parse(args: List[String]): Unit = args match { - case ("--host" | "-h") :: value :: tail => - Utils.checkHost(value) - host = value - parse(tail) - - case ("--port" | "-p") :: IntParam(value) :: tail => - port = value - parse(tail) - - case ("--webui-port") :: IntParam(value) :: tail => - webUiPort = value - parse(tail) - - case ("--zk" | "-z") :: value :: tail => - zookeeperUrl = Some(value) - parse(tail) - - case ("--master" | "-m") :: value :: tail => - if (!value.startsWith("mesos://")) { - // scalastyle:off println - MesosClusterDispatcher.printStream - .println("Cluster dispatcher only supports mesos (uri begins with mesos://)") - // scalastyle:on println - MesosClusterDispatcher.exitFn(1) - } - masterUrl = value.stripPrefix("mesos://") - parse(tail) - - case ("--name") :: value :: tail => - name = value - parse(tail) - - case ("--properties-file") :: value :: tail => - propertiesFile = value - parse(tail) - - case ("--conf") :: value :: tail => - val (k, v) = SparkSubmitUtils.parseSparkConfProperty(value) - confProperties(k) = v - parse(tail) - - case ("--help") :: tail => - printUsageAndExit(0) - - case ("--verbose") :: tail => - verbose = true - parse(tail) - - case Nil => - if (Option(masterUrl).isEmpty) { - // scalastyle:off println - MesosClusterDispatcher.printStream.println("--master is required") - // scalastyle:on println - printUsageAndExit(1) - } - - case value => - // scalastyle:off println - MesosClusterDispatcher.printStream.println(s"Unrecognized option: '${value.head}'") - // scalastyle:on println - printUsageAndExit(1) - } - - private def printUsageAndExit(exitCode: Int): Unit = { - val outStream = MesosClusterDispatcher.printStream - - // scalastyle:off println - outStream.println( - "Usage: MesosClusterDispatcher [options]\n" + - "\n" + - "Options:\n" + - " -h HOST, --host HOST Hostname to listen on\n" + - " --help Show this help message and exit.\n" + - " --verbose, Print additional debug output.\n" + - " -p PORT, --port PORT Port to listen on (default: 7077)\n" + - " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + - " --name NAME Framework name to show in Mesos UI\n" + - " -m --master MASTER URI for connecting to Mesos master\n" + - " -z --zk ZOOKEEPER Comma delimited URLs for connecting to \n" + - " Zookeeper for persistence\n" + - " --properties-file FILE Path to a custom Spark properties file.\n" + - " Default is conf/spark-defaults.conf \n" + - " --conf PROP=VALUE Arbitrary Spark configuration property.\n" + - " Takes precedence over defined properties in properties-file.") - // scalastyle:on println - MesosClusterDispatcher.exitFn(exitCode) - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala deleted file mode 100644 index d4c7022f006a1..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import java.util.Date - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.Command -import org.apache.spark.scheduler.cluster.mesos.MesosClusterRetryState - -/** - * Describes a Spark driver that is submitted from the - * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]], to be launched by - * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]]. - * @param jarUrl URL to the application jar - * @param mem Amount of memory for the driver - * @param cores Number of cores for the driver - * @param supervise Supervise the driver for long running app - * @param command The command to launch the driver. - * @param schedulerProperties Extra properties to pass the Mesos scheduler - */ -private[spark] class MesosDriverDescription( - val name: String, - val jarUrl: String, - val mem: Int, - val cores: Double, - val supervise: Boolean, - val command: Command, - schedulerProperties: Map[String, String], - val submissionId: String, - val submissionDate: Date, - val retryState: Option[MesosClusterRetryState] = None) - extends Serializable { - - val conf = new SparkConf(false) - schedulerProperties.foreach {case (k, v) => conf.set(k, v)} - - def copy( - name: String = name, - jarUrl: String = jarUrl, - mem: Int = mem, - cores: Double = cores, - supervise: Boolean = supervise, - command: Command = command, - schedulerProperties: SparkConf = conf, - submissionId: String = submissionId, - submissionDate: Date = submissionDate, - retryState: Option[MesosClusterRetryState] = retryState): MesosDriverDescription = { - - new MesosDriverDescription(name, jarUrl, mem, cores, supervise, command, conf.getAll.toMap, - submissionId, submissionDate, retryState) - } - - override def toString: String = s"MesosDriverDescription (${command.mainClass})" -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala deleted file mode 100644 index e6870feb92863..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import java.nio.ByteBuffer -import java.util.concurrent.{ConcurrentHashMap, TimeUnit} - -import scala.jdk.CollectionConverters._ - -import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.deploy.ExternalShuffleService -import org.apache.spark.deploy.mesos.config._ -import org.apache.spark.internal.Logging -import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} -import org.apache.spark.network.shuffle.ExternalBlockHandler -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage -import org.apache.spark.network.shuffle.protocol.mesos.{RegisterDriver, ShuffleServiceHeartbeat} -import org.apache.spark.network.util.TransportConf -import org.apache.spark.util.ThreadUtils - -/** - * An RPC endpoint that receives registration requests from Spark drivers running on Mesos. - * It detects driver termination and calls the cleanup callback to [[ExternalShuffleService]]. - */ -private[mesos] class MesosExternalBlockHandler( - transportConf: TransportConf, - cleanerIntervalS: Long) - extends ExternalBlockHandler(transportConf, null) with Logging { - - ThreadUtils.newDaemonSingleThreadScheduledExecutor("shuffle-cleaner-watcher") - .scheduleAtFixedRate(new CleanerThread(), 0, cleanerIntervalS, TimeUnit.SECONDS) - - // Stores a map of app id to app state (timeout value and last heartbeat) - private val connectedApps = new ConcurrentHashMap[String, AppState]() - - protected override def handleMessage( - message: BlockTransferMessage, - client: TransportClient, - callback: RpcResponseCallback): Unit = { - message match { - case RegisterDriverParam(appId, appState) => - val address = client.getSocketAddress - val timeout = appState.heartbeatTimeout - logInfo(s"Received registration request from app $appId (remote address $address, " + - s"heartbeat timeout $timeout ms).") - if (connectedApps.containsKey(appId)) { - logWarning(s"Received a registration request from app $appId, but it was already " + - s"registered") - } - connectedApps.put(appId, appState) - callback.onSuccess(ByteBuffer.allocate(0)) - case Heartbeat(appId) => - val address = client.getSocketAddress - Option(connectedApps.get(appId)) match { - case Some(existingAppState) => - logTrace(s"Received ShuffleServiceHeartbeat from app '$appId' (remote " + - s"address $address).") - existingAppState.lastHeartbeat = System.nanoTime() - case None => - logWarning(s"Received ShuffleServiceHeartbeat from an unknown app (remote " + - s"address $address, appId '$appId').") - } - case _ => super.handleMessage(message, client, callback) - } - } - - /** An extractor object for matching [[RegisterDriver]] message. */ - private object RegisterDriverParam { - def unapply(r: RegisterDriver): Option[(String, AppState)] = - Some((r.getAppId, new AppState(r.getHeartbeatTimeoutMs, System.nanoTime()))) - } - - private object Heartbeat { - def unapply(h: ShuffleServiceHeartbeat): Option[String] = Some(h.getAppId) - } - - private class AppState(val heartbeatTimeout: Long, @volatile var lastHeartbeat: Long) - - private class CleanerThread extends Runnable { - override def run(): Unit = { - val now = System.nanoTime() - connectedApps.asScala.foreach { case (appId, appState) => - if (now - appState.lastHeartbeat > appState.heartbeatTimeout * 1000 * 1000) { - logInfo(s"Application $appId timed out. Removing shuffle files.") - connectedApps.remove(appId) - applicationRemoved(appId, true) - } - } - } - } -} - -/** - * A wrapper of [[ExternalShuffleService]] that provides an additional endpoint for drivers - * to associate with. This allows the shuffle service to detect when a driver is terminated - * and can clean up the associated shuffle files. - */ -private[mesos] class MesosExternalShuffleService(conf: SparkConf, securityManager: SecurityManager) - extends ExternalShuffleService(conf, securityManager) { - - protected override def newShuffleBlockHandler( - conf: TransportConf): ExternalBlockHandler = { - val cleanerIntervalS = this.conf.get(SHUFFLE_CLEANER_INTERVAL_S) - new MesosExternalBlockHandler(conf, cleanerIntervalS) - } -} - -private[spark] object MesosExternalShuffleService extends Logging { - - def main(args: Array[String]): Unit = { - ExternalShuffleService.main(args, - (conf: SparkConf, sm: SecurityManager) => new MesosExternalShuffleService(conf, sm)) - } -} - - diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala deleted file mode 100644 index 38df43d71b897..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala +++ /dev/null @@ -1,420 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import java.util.concurrent.TimeUnit - -import org.apache.spark.internal.config.ConfigBuilder - -package object config { - - private[spark] class MesosSecretConfig private[config](taskType: String) { - private[spark] val SECRET_NAMES = - ConfigBuilder(s"spark.mesos.$taskType.secret.names") - .doc("A comma-separated list of secret reference names. Consult the Mesos Secret " + - "protobuf for more information.") - .version("2.3.0") - .stringConf - .toSequence - .createOptional - - private[spark] val SECRET_VALUES = - ConfigBuilder(s"spark.mesos.$taskType.secret.values") - .doc("A comma-separated list of secret values.") - .version("2.3.0") - .stringConf - .toSequence - .createOptional - - private[spark] val SECRET_ENVKEYS = - ConfigBuilder(s"spark.mesos.$taskType.secret.envkeys") - .doc("A comma-separated list of the environment variables to contain the secrets." + - "The environment variable will be set on the driver.") - .version("2.3.0") - .stringConf - .toSequence - .createOptional - - private[spark] val SECRET_FILENAMES = - ConfigBuilder(s"spark.mesos.$taskType.secret.filenames") - .doc("A comma-separated list of file paths secret will be written to. Consult the Mesos " + - "Secret protobuf for more information.") - .version("2.3.0") - .stringConf - .toSequence - .createOptional - } - - private[spark] val CREDENTIAL_PRINCIPAL = - ConfigBuilder("spark.mesos.principal") - .doc("Name of the Kerberos principal to authenticate Spark to Mesos.") - .version("1.5.0") - .stringConf - .createOptional - - private[spark] val CREDENTIAL_PRINCIPAL_FILE = - ConfigBuilder("spark.mesos.principal.file") - .doc("The path of file which contains the name of the Kerberos principal " + - "to authenticate Spark to Mesos.") - .version("2.4.0") - .stringConf - .createOptional - - private[spark] val CREDENTIAL_SECRET = - ConfigBuilder("spark.mesos.secret") - .doc("The secret value to authenticate Spark to Mesos.") - .version("1.5.0") - .stringConf - .createOptional - - private[spark] val CREDENTIAL_SECRET_FILE = - ConfigBuilder("spark.mesos.secret.file") - .doc("The path of file which contains the secret value to authenticate Spark to Mesos.") - .version("2.4.0") - .stringConf - .createOptional - - /* Common app configuration. */ - - private[spark] val SHUFFLE_CLEANER_INTERVAL_S = - ConfigBuilder("spark.shuffle.cleaner.interval") - .version("2.0.0") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("30s") - - private[spark] val DISPATCHER_WEBUI_URL = - ConfigBuilder("spark.mesos.dispatcher.webui.url") - .doc("Set the Spark Mesos dispatcher webui_url for interacting with the " + - "framework. If unset it will point to Spark's internal web UI.") - .version("2.0.0") - .stringConf - .createOptional - - private[spark] val HISTORY_SERVER_URL = - ConfigBuilder("spark.mesos.dispatcher.historyServer.url") - .doc("Set the URL of the history server. The dispatcher will then " + - "link each driver to its entry in the history server.") - .version("2.1.0") - .stringConf - .createOptional - - private[spark] val DISPATCHER_QUEUE = - ConfigBuilder("spark.mesos.dispatcher.queue") - .doc("Set the name of the dispatcher queue to which the application is submitted. " + - "The specified queue must be added to the dispatcher " + - "with \"spark.mesos.dispatcher.queue.[QueueName]\". If no queue is specified, then " + - "the application is submitted to the \"default\" queue with 0.0 priority.") - .version("3.1.0") - .stringConf - .createWithDefaultString("default") - - private[spark] val DRIVER_LABELS = - ConfigBuilder("spark.mesos.driver.labels") - .doc("Mesos labels to add to the driver. Labels are free-form key-value pairs. Key-value " + - "pairs should be separated by a colon, and commas used to list more than one." + - "Ex. key:value,key2:value2") - .version("2.3.0") - .stringConf - .createOptional - - private[spark] val DRIVER_WEBUI_URL = - ConfigBuilder("spark.mesos.driver.webui.url") - .doc("Set the Spark Mesos driver webui_url for interacting with the framework. " + - "If unset it will point to Spark's internal web UI.") - .version("2.0.0") - .stringConf - .createOptional - - private[spark] val driverSecretConfig = new MesosSecretConfig("driver") - - private[spark] val executorSecretConfig = new MesosSecretConfig("executor") - - private[spark] val DRIVER_FAILOVER_TIMEOUT = - ConfigBuilder("spark.mesos.driver.failoverTimeout") - .doc("Amount of time in seconds that the master will wait to hear from the driver, " + - "during a temporary disconnection, before tearing down all the executors.") - .version("2.3.0") - .doubleConf - .createWithDefault(0.0) - - private[spark] val NETWORK_NAME = - ConfigBuilder("spark.mesos.network.name") - .doc("Attach containers to the given named network. If this job is launched " + - "in cluster mode, also launch the driver in the given named network.") - .version("2.1.0") - .stringConf - .createOptional - - private[spark] val NETWORK_LABELS = - ConfigBuilder("spark.mesos.network.labels") - .doc("Network labels to pass to CNI plugins. This is a comma-separated list " + - "of key-value pairs, where each key-value pair has the format key:value. " + - "Example: key1:val1,key2:val2") - .version("2.3.0") - .stringConf - .createOptional - - private[spark] val DRIVER_CONSTRAINTS = - ConfigBuilder("spark.mesos.driver.constraints") - .doc("Attribute based constraints on mesos resource offers. Applied by the dispatcher " + - "when launching drivers. Default is to accept all offers with sufficient resources.") - .version("2.2.1") - .stringConf - .createWithDefault("") - - private[spark] val DRIVER_FRAMEWORK_ID = - ConfigBuilder("spark.mesos.driver.frameworkId") - .version("2.1.0") - .stringConf - .createOptional - - private[spark] val DRIVER_MEMORY_OVERHEAD = - ConfigBuilder("spark.mesos.driver.memoryOverhead") - .doc("The amount of additional memory, specified in MB, to be allocated to the driver. " + - "By default, the overhead will be larger of either 384 or 10% of spark.driver.memory. " + - "Only applies to cluster mode.") - .intConf - .createOptional - - private[spark] val EXECUTOR_URI = - ConfigBuilder("spark.executor.uri").version("0.8.0").stringConf.createOptional - - private[spark] val PROXY_BASE_URL = - ConfigBuilder("spark.mesos.proxy.baseURL").version("2.3.0").stringConf.createOptional - - private[spark] val COARSE_MODE = - ConfigBuilder("spark.mesos.coarse") - .doc("If set to true, runs over Mesos clusters in \"coarse-grained\" sharing mode, where " + - "Spark acquires one long-lived Mesos task on each machine. If set to false, runs over " + - "Mesos cluster in \"fine-grained\" sharing mode, where one Mesos task is created per " + - "Spark task.") - .version("0.6.0") - .booleanConf - .createWithDefault(true) - - private[spark] val COARSE_SHUTDOWN_TIMEOUT = - ConfigBuilder("spark.mesos.coarse.shutdownTimeout") - .version("2.0.0") - .timeConf(TimeUnit.MILLISECONDS) - .checkValue(_ >= 0, s"spark.mesos.coarse.shutdownTimeout must be >= 0") - .createWithDefaultString("10s") - - private[spark] val MAX_DRIVERS = - ConfigBuilder("spark.mesos.maxDrivers").version("1.4.0").intConf.createWithDefault(200) - - private[spark] val RETAINED_DRIVERS = - ConfigBuilder("spark.mesos.retainedDrivers").version("1.4.0").intConf.createWithDefault(200) - - private[spark] val CLUSTER_RETRY_WAIT_MAX_SECONDS = - ConfigBuilder("spark.mesos.cluster.retry.wait.max") - .version("1.4.0") - .intConf - .createWithDefault(60) // 1 minute - - private[spark] val ENABLE_FETCHER_CACHE = - ConfigBuilder("spark.mesos.fetcherCache.enable") - .doc("If set to true, all URIs (example: `spark.executor.uri`, `spark.mesos.uris`) will be " + - "cached by the Mesos Fetcher Cache.") - .version("2.1.0") - .booleanConf - .createWithDefault(false) - - private[spark] val APP_JAR_LOCAL_RESOLUTION_MODE = - ConfigBuilder("spark.mesos.appJar.local.resolution.mode") - .doc("Provides support for the `local:///` scheme to reference the app jar resource in " + - "cluster mode. If user uses a local resource (`local:///path/to/jar`) and the config " + - "option is not used it defaults to `host` e.g. the mesos fetcher tries to get the " + - "resource from the host's file system. If the value is unknown it prints a warning msg " + - "in the dispatcher logs and defaults to `host`. If the value is `container` then spark " + - "submit in the container will use the jar in the container's path: `/path/to/jar`.") - .version("2.4.0") - .stringConf - .checkValues(Set("host", "container")) - .createWithDefault("host") - - private[spark] val REJECT_OFFER_DURATION = - ConfigBuilder("spark.mesos.rejectOfferDuration") - .doc("Time to consider unused resources refused, serves as a fallback of " + - "`spark.mesos.rejectOfferDurationForUnmetConstraints`, " + - "`spark.mesos.rejectOfferDurationForReachedMaxCores`.") - .version("2.2.0") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("120s") - - private[spark] val REJECT_OFFER_DURATION_FOR_UNMET_CONSTRAINTS = - ConfigBuilder("spark.mesos.rejectOfferDurationForUnmetConstraints") - .doc("Time to consider unused resources refused with unmet constraints.") - .version("1.6.0") - .timeConf(TimeUnit.SECONDS) - .createOptional - - private[spark] val REJECT_OFFER_DURATION_FOR_REACHED_MAX_CORES = - ConfigBuilder("spark.mesos.rejectOfferDurationForReachedMaxCores") - .doc("Time to consider unused resources refused when maximum number of cores " + - "`spark.cores.max` is reached.") - .version("2.0.0") - .timeConf(TimeUnit.SECONDS) - .createOptional - - private[spark] val URIS_TO_DOWNLOAD = - ConfigBuilder("spark.mesos.uris") - .doc("A comma-separated list of URIs to be downloaded to the sandbox when driver or " + - "executor is launched by Mesos. This applies to both coarse-grained and fine-grained " + - "mode.") - .version("1.5.0") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val EXECUTOR_HOME = - ConfigBuilder("spark.mesos.executor.home") - .doc("Set the directory in which Spark is installed on the executors in Mesos. " + - "By default, the executors will simply use the driver's Spark home directory, which may " + - "not be visible to them. Note that this is only relevant if a Spark binary package is " + - "not specified through `spark.executor.uri`.") - .version("1.1.1") - .stringConf - .createOptional - - private[spark] val EXECUTOR_CORES = - ConfigBuilder("spark.mesos.mesosExecutor.cores") - .doc("(Fine-grained mode only) Number of cores to give each Mesos executor. This does not " + - "include the cores used to run the Spark tasks. In other words, even if no Spark task " + - "is being run, each Mesos executor will occupy the number of cores configured here. " + - "The value can be a floating point number.") - .version("1.4.0") - .doubleConf - .createWithDefault(1.0) - - private[spark] val EXTRA_CORES_PER_EXECUTOR = - ConfigBuilder("spark.mesos.extra.cores") - .doc("Set the extra number of cores for an executor to advertise. This does not result in " + - "more cores allocated. It instead means that an executor will \"pretend\" it has more " + - "cores, so that the driver will send it more tasks. Use this to increase parallelism. " + - "This setting is only used for Mesos coarse-grained mode.") - .version("0.6.0") - .intConf - .createWithDefault(0) - - private[spark] val EXECUTOR_MEMORY_OVERHEAD = - ConfigBuilder("spark.mesos.executor.memoryOverhead") - .doc("The amount of additional memory, specified in MiB, to be allocated per executor. " + - "By default, the overhead will be larger of either 384 or 10% of " + - "`spark.executor.memory`. If set, the final overhead will be this value.") - .version("1.1.1") - .intConf - .createOptional - - private[spark] val EXECUTOR_DOCKER_IMAGE = - ConfigBuilder("spark.mesos.executor.docker.image") - .doc("Set the name of the docker image that the Spark executors will run in. The selected " + - "image must have Spark installed, as well as a compatible version of the Mesos library. " + - "The installed path of Spark in the image can be specified with " + - "`spark.mesos.executor.home`; the installed path of the Mesos library can be specified " + - "with `spark.executorEnv.MESOS_NATIVE_JAVA_LIBRARY`.") - .version("1.4.0") - .stringConf - .createOptional - - private[spark] val EXECUTOR_DOCKER_FORCE_PULL_IMAGE = - ConfigBuilder("spark.mesos.executor.docker.forcePullImage") - .doc("Force Mesos agents to pull the image specified in " + - "`spark.mesos.executor.docker.image`. By default Mesos agents will not pull images they " + - "already have cached.") - .version("2.1.0") - .booleanConf - .createOptional - - private[spark] val EXECUTOR_DOCKER_PORT_MAPS = - ConfigBuilder("spark.mesos.executor.docker.portmaps") - .version("1.4.0") - .stringConf - .toSequence - .createOptional - - private[spark] val EXECUTOR_DOCKER_PARAMETERS = - ConfigBuilder("spark.mesos.executor.docker.parameters") - .doc("Set the list of custom parameters which will be passed into the `docker run` " + - "command when launching the Spark executor on Mesos using the docker containerizer. " + - "The format of this property is a list of key/value pairs which pair looks key1=value1.") - .version("2.2.0") - .stringConf - .toSequence - .createOptional - - private[spark] val EXECUTOR_DOCKER_VOLUMES = - ConfigBuilder("spark.mesos.executor.docker.volumes") - .doc("Set the list of volumes which will be mounted into the Docker image, which was set " + - "using `spark.mesos.executor.docker.image`. The format of this property is a list of " + - "mappings following the form passed to `docker run -v`. That is they take the form: " + - "`[host_path:]container_path[:ro|:rw]`") - .version("1.4.0") - .stringConf - .toSequence - .createOptional - - private[spark] val MAX_GPUS = - ConfigBuilder("spark.mesos.gpus.max") - .doc("Set the maximum number GPU resources to acquire for this job. Note that executors " + - "will still launch when no GPU resources are found since this configuration is just an " + - "upper limit and not a guaranteed amount.") - .version("2.1.0") - .intConf - .createWithDefault(0) - - private[spark] val TASK_LABELS = - ConfigBuilder("spark.mesos.task.labels") - .doc("Set the Mesos labels to add to each task. Labels are free-form key-value pairs. " + - "Key-value pairs should be separated by a colon, and commas used to list more than one. " + - "If your label includes a colon or comma, you can escape it with a backslash. " + - "Ex. key:value,key2:a\\:b.") - .version("2.2.0") - .stringConf - .createWithDefault("") - - private[spark] val CONSTRAINTS = - ConfigBuilder("spark.mesos.constraints") - .doc("Attribute-based constraints on mesos resource offers. By default, all resource " + - "offers will be accepted. This setting applies only to executors. Refer to Mesos " + - "Attributes & Resources doc for more information on attributes.") - .version("1.5.0") - .stringConf - .createWithDefault("") - - private[spark] val CONTAINERIZER = - ConfigBuilder("spark.mesos.containerizer") - .doc("This only affects docker containers, and must be one of \"docker\" or \"mesos\". " + - "Mesos supports two types of containerizers for docker: the \"docker\" containerizer, " + - "and the preferred \"mesos\" containerizer. " + - "Read more here: http://mesos.apache.org/documentation/latest/container-image/") - .version("2.1.0") - .stringConf - .checkValues(Set("docker", "mesos")) - .createWithDefault("docker") - - private[spark] val ROLE = - ConfigBuilder("spark.mesos.role") - .doc("Set the role of this Spark framework for Mesos. Roles are used in Mesos for " + - "reservations and resource weight sharing.") - .version("1.5.0") - .stringConf - .createOptional - - private[spark] val DRIVER_ENV_PREFIX = "spark.mesos.driverEnv." - private[spark] val DISPATCHER_DRIVER_DEFAULT_PREFIX = "spark.mesos.dispatcher.driverDefault." -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala deleted file mode 100644 index 97ef153177674..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos.ui - -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node - -import org.apache.spark.deploy.Command -import org.apache.spark.deploy.mesos.{config, MesosDriverDescription} -import org.apache.spark.scheduler.cluster.mesos.{MesosClusterRetryState, MesosClusterSubmissionState} -import org.apache.spark.ui.{UIUtils, WebUIPage} - -private[ui] class DriverPage(parent: MesosClusterUI) extends WebUIPage("driver") { - - override def render(request: HttpServletRequest): Seq[Node] = { - val driverId = request.getParameter("id") - require(driverId != null && driverId.nonEmpty, "Missing id parameter") - - val state = parent.scheduler.getDriverState(driverId) - if (state.isEmpty) { - val content = -
-

Cannot find driver {driverId}

-
- return UIUtils.basicSparkPage(request, content, s"Details for Job $driverId") - } - val driverState = state.get - val driverHeaders = Seq("Driver property", "Value") - val schedulerHeaders = Seq("Scheduler property", "Value") - val commandEnvHeaders = Seq("Command environment variable", "Value") - val launchedHeaders = Seq("Launched property", "Value") - val commandHeaders = Seq("Command property", "Value") - val retryHeaders = Seq("Last failed status", "Next retry time", "Retry count") - val driverDescription = Iterable.apply(driverState.description) - val submissionState = Iterable.apply(driverState.submissionState) - val command = Iterable.apply(driverState.description.command) - val schedulerProperties = Iterable.apply(driverState.description.conf.getAll.toMap) - val commandEnv = Iterable.apply(driverState.description.command.environment) - val driverTable = - UIUtils.listingTable(driverHeaders, driverRow, driverDescription) - val commandTable = - UIUtils.listingTable(commandHeaders, commandRow, command) - val commandEnvTable = - UIUtils.listingTable(commandEnvHeaders, propertiesRow, commandEnv) - val schedulerTable = - UIUtils.listingTable(schedulerHeaders, propertiesRow, schedulerProperties) - val launchedTable = - UIUtils.listingTable(launchedHeaders, launchedRow, submissionState) - val retryTable = - UIUtils.listingTable( - retryHeaders, retryRow, Iterable.apply(driverState.description.retryState)) - val content = -

Driver state information for driver id {driverId}

- Back to Drivers -
-
-

Driver state: {driverState.state}

-

Driver properties

- {driverTable} -

Driver command

- {commandTable} -

Driver command environment

- {commandEnvTable} -

Scheduler properties

- {schedulerTable} -

Launched state

- {launchedTable} -

Retry state

- {retryTable} -
-
; - - UIUtils.basicSparkPage(request, content, s"Details for Job $driverId") - } - - private def launchedRow(submissionState: Option[MesosClusterSubmissionState]): Seq[Node] = { - submissionState.map { state => - - Mesos Agent ID - {state.agentId.getValue} - - - Mesos Task ID - {state.taskId.getValue} - - - Launch Time - {UIUtils.formatDate(state.startDate)} - - - Finish Time - {state.finishDate.map(_.toString).getOrElse("")} - - - Last Task Status - {state.mesosTaskStatus.map(_.toString).getOrElse("")} - - }.getOrElse(Seq.empty[Node]) - } - - private def propertiesRow(properties: collection.Map[String, String]): Seq[Node] = { - properties.map { case (k, v) => - - {k}{v} - - }.toSeq - } - - private def commandRow(command: Command): Seq[Node] = { - - Main class{command.mainClass} - - - Arguments{command.arguments.mkString(" ")} - - - Class path entries{command.classPathEntries.mkString(" ")} - - - Java options{command.javaOpts.mkString((" "))} - - - Library path entries{command.libraryPathEntries.mkString((" "))} - - } - - private def driverRow(driver: MesosDriverDescription): Seq[Node] = { - - Name{driver.name} - - - Id{driver.submissionId} - - - Cores{driver.cores} - - - Memory{driver.mem} - - - Queue - - {driver.conf.get( - "spark.mesos.dispatcher.queue", config.DISPATCHER_QUEUE.defaultValueString)} - - - - Submitted{UIUtils.formatDate(driver.submissionDate)} - - - Supervise{driver.supervise} - - } - - private def retryRow(retryState: Option[MesosClusterRetryState]): Seq[Node] = { - retryState.map { state => - - - {state.lastFailureStatus} - - - {state.nextRetry} - - - {state.retries} - - - }.getOrElse(Seq.empty[Node]) - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala deleted file mode 100644 index 5c62ddb37684d..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos.ui - -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node - -import org.apache.mesos.Protos.TaskStatus - -import org.apache.spark.deploy.mesos.{config, MesosDriverDescription} -import org.apache.spark.deploy.mesos.config._ -import org.apache.spark.scheduler.cluster.mesos.MesosClusterSubmissionState -import org.apache.spark.ui.{UIUtils, WebUIPage} - -private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") { - private val historyServerURL = parent.conf.get(HISTORY_SERVER_URL) - - def render(request: HttpServletRequest): Seq[Node] = { - val state = parent.scheduler.getSchedulerState() - - val driverHeader = Seq("Driver ID") - val historyHeader = historyServerURL.map(url => Seq("History")).getOrElse(Nil) - val submissionHeader = Seq("Queue", "Submit Date", "Main Class", "Driver Resources") - val sandboxHeader = Seq("Sandbox") - - val queuedHeaders = driverHeader ++ submissionHeader - val driverHeaders = driverHeader ++ historyHeader ++ submissionHeader ++ - Seq("Start Date", "Mesos Agent ID", "State") ++ sandboxHeader - val retryHeaders = Seq("Driver ID", "Submit Date", "Description") ++ - Seq("Last Failed Status", "Next Retry Time", "Attempt Count") - val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) - val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) - val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers) - val retryTable = UIUtils.listingTable(retryHeaders, retryRow, state.pendingRetryDrivers) - val content = -

Mesos Framework ID: {state.frameworkId}

-
-
-

Queued Drivers:

- {queuedTable} -

Launched Drivers:

- {launchedTable} -

Finished Drivers:

- {finishedTable} -

Supervise drivers waiting for retry:

- {retryTable} -
-
; - UIUtils.basicSparkPage(request, content, "Spark Drivers for Mesos cluster") - } - - private def queuedRow(submission: MesosDriverDescription): Seq[Node] = { - val id = submission.submissionId - - {id} - - {submission.conf.get( - "spark.mesos.dispatcher.queue", config.DISPATCHER_QUEUE.defaultValueString)} - - {UIUtils.formatDate(submission.submissionDate)} - {submission.command.mainClass} - cpus: {submission.cores}, mem: {submission.mem} - - } - - private def driverRow(state: MesosClusterSubmissionState): Seq[Node] = { - val id = state.driverDescription.submissionId - val proxy = parent.conf.get(PROXY_BASE_URL) - - val sandboxCol = if (proxy.isDefined) { - val clusterSchedulerId = parent.scheduler.getSchedulerState().frameworkId - val sandBoxUri = s"${proxy.get}/#/agents/${state.agentId.getValue}/frameworks/" + - s"${clusterSchedulerId}/executors/${id}/browse" - Sandbox - } else { - " " - } - - val historyCol = if (historyServerURL.isDefined) { - - - {state.frameworkId} - - - } else Nil - - - {id} - {historyCol} - - {state.driverDescription.conf.get( - "spark.mesos.dispatcher.queue", config.DISPATCHER_QUEUE.defaultValueString)} - - {UIUtils.formatDate(state.driverDescription.submissionDate)} - {state.driverDescription.command.mainClass} - cpus: {state.driverDescription.cores}, mem: {state.driverDescription.mem} - {UIUtils.formatDate(state.startDate)} - {state.agentId.getValue} - {stateString(state.mesosTaskStatus)} - {sandboxCol} - - } - - private def retryRow(submission: MesosDriverDescription): Seq[Node] = { - val id = submission.submissionId - - {id} - {UIUtils.formatDate(submission.submissionDate)} - {submission.command.mainClass} - {submission.retryState.get.lastFailureStatus} - {submission.retryState.get.nextRetry} - {submission.retryState.get.retries} - - } - - private def stateString(status: Option[TaskStatus]): String = { - if (status.isEmpty) { - return "" - } - val sb = new StringBuilder - val s = status.get - sb.append(s"State: ${s.getState}") - if (status.get.hasMessage) { - sb.append(s", Message: ${s.getMessage}") - } - if (status.get.hasHealthy) { - sb.append(s", Healthy: ${s.getHealthy}") - } - if (status.get.hasSource) { - sb.append(s", Source: ${s.getSource}") - } - if (status.get.hasReason) { - sb.append(s", Reason: ${s.getReason}") - } - if (status.get.hasTimestamp) { - sb.append(s", Time: ${s.getTimestamp}") - } - sb.toString() - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala deleted file mode 100644 index e260fb8e25f4c..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos.ui - -import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler -import org.apache.spark.ui.{SparkUI, WebUI} - -/** - * UI that displays driver results from the [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]] - */ -private[spark] class MesosClusterUI( - securityManager: SecurityManager, - port: Int, - val conf: SparkConf, - dispatcherPublicAddress: String, - val scheduler: MesosClusterScheduler) - extends WebUI(securityManager, securityManager.getSSLOptions("mesos"), port, conf) { - - initialize() - - def activeWebUiUrl: String = "http://" + dispatcherPublicAddress + ":" + boundPort - - override def initialize(): Unit = { - attachPage(new MesosClusterPage(this)) - attachPage(new DriverPage(this)) - addStaticHandler(MesosClusterUI.STATIC_RESOURCE_DIR) - } -} - -private object MesosClusterUI { - val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala deleted file mode 100644 index 9e4187837b680..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ /dev/null @@ -1,176 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.rest.mesos - -import java.io.File -import java.text.SimpleDateFormat -import java.util.{Date, Locale} -import java.util.concurrent.atomic.AtomicLong -import javax.servlet.http.HttpServletResponse - -import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} -import org.apache.spark.deploy.Command -import org.apache.spark.deploy.mesos.MesosDriverDescription -import org.apache.spark.deploy.rest._ -import org.apache.spark.internal.config -import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler -import org.apache.spark.util.Utils - -/** - * A server that responds to requests submitted by the [[RestSubmissionClient]]. - * All requests are forwarded to - * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]]. - * This is intended to be used in Mesos cluster mode only. - * For more details about the REST submission please refer to [[RestSubmissionServer]] javadocs. - */ -private[spark] class MesosRestServer( - host: String, - requestedPort: Int, - masterConf: SparkConf, - scheduler: MesosClusterScheduler) - extends RestSubmissionServer(host, requestedPort, masterConf) { - - protected override val submitRequestServlet = - new MesosSubmitRequestServlet(scheduler, masterConf) - protected override val killRequestServlet = - new MesosKillRequestServlet(scheduler, masterConf) - protected override val statusRequestServlet = - new MesosStatusRequestServlet(scheduler, masterConf) -} - -private[mesos] class MesosSubmitRequestServlet( - scheduler: MesosClusterScheduler, - conf: SparkConf) - extends SubmitRequestServlet { - - private val DEFAULT_SUPERVISE = false - private val DEFAULT_MEMORY = Utils.DEFAULT_DRIVER_MEM_MB // mb - private val DEFAULT_CORES = 1.0 - - private val nextDriverNumber = new AtomicLong(0) - // For application IDs - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) - private def newDriverId(submitDate: Date): String = - f"driver-${createDateFormat.format(submitDate)}-${nextDriverNumber.incrementAndGet()}%04d" - - // These defaults copied from YARN - private val MEMORY_OVERHEAD_FACTOR = 0.10 - private val MEMORY_OVERHEAD_MIN = 384 - - /** - * Build a driver description from the fields specified in the submit request. - * - * This involves constructing a command that launches a mesos framework for the job. - * This does not currently consider fields used by python applications since python - * is not supported in mesos cluster mode yet. - */ - private def buildDriverDescription(request: CreateSubmissionRequest): MesosDriverDescription = { - // Required fields, including the main class because python is not yet supported - val appResource = Option(request.appResource).getOrElse { - throw new SubmitRestMissingFieldException("Application jar 'appResource' is missing.") - } - val mainClass = Option(request.mainClass).getOrElse { - throw new SubmitRestMissingFieldException("Main class 'mainClass' is missing.") - } - val appArgs = Option(request.appArgs).getOrElse { - throw new SubmitRestMissingFieldException("Application arguments 'appArgs' are missing.") - } - val environmentVariables = Option(request.environmentVariables).getOrElse { - throw new SubmitRestMissingFieldException("Environment variables 'environmentVariables' " + - "are missing.") - } - - // Optional fields - val sparkProperties = request.sparkProperties - val driverDefaultJavaOptions = sparkProperties.get(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS) - val driverExtraJavaOptions = sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key) - val driverExtraClassPath = sparkProperties.get(config.DRIVER_CLASS_PATH.key) - val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) - val superviseDriver = sparkProperties.get(config.DRIVER_SUPERVISE.key) - val driverMemory = sparkProperties.get(config.DRIVER_MEMORY.key) - val driverMemoryOverhead = sparkProperties.get(config.DRIVER_MEMORY_OVERHEAD.key) - val driverMemoryOverheadFactor = sparkProperties.get(config.DRIVER_MEMORY_OVERHEAD_FACTOR.key) - val driverCores = sparkProperties.get(config.DRIVER_CORES.key) - val name = request.sparkProperties.getOrElse("spark.app.name", mainClass) - - // Construct driver description - val conf = new SparkConf(false).setAll(sparkProperties) - val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) - val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) - val defaultJavaOpts = driverDefaultJavaOptions.map(Utils.splitCommandString) - .getOrElse(Seq.empty) - val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) - val sparkJavaOpts = Utils.sparkJavaOpts(conf) - val javaOpts = sparkJavaOpts ++ defaultJavaOpts ++ extraJavaOpts - val command = new Command( - mainClass, appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts) - val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) - val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) - val actualDriverMemoryFactor = driverMemoryOverheadFactor.map(_.toDouble).getOrElse( - MEMORY_OVERHEAD_FACTOR) - val actualDriverMemoryOverhead = driverMemoryOverhead.map(_.toInt).getOrElse( - math.max((actualDriverMemoryFactor * actualDriverMemory).toInt, MEMORY_OVERHEAD_MIN)) - val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES) - val submitDate = new Date() - val submissionId = newDriverId(submitDate) - - new MesosDriverDescription( - name, appResource, actualDriverMemory + actualDriverMemoryOverhead, actualDriverCores, - actualSuperviseDriver, command, request.sparkProperties, submissionId, submitDate) - } - - protected override def handleSubmit( - requestMessageJson: String, - requestMessage: SubmitRestProtocolMessage, - responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { - requestMessage match { - case submitRequest: CreateSubmissionRequest => - val driverDescription = buildDriverDescription(submitRequest) - val s = scheduler.submitDriver(driverDescription) - s.serverSparkVersion = sparkVersion - val unknownFields = findUnknownFields(requestMessageJson, requestMessage) - if (unknownFields.nonEmpty) { - // If there are fields that the server does not know about, warn the client - s.unknownFields = unknownFields - } - s - case unexpected => - responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError(s"Received message of unexpected type ${unexpected.messageType}.") - } - } -} - -private[mesos] class MesosKillRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) - extends KillRequestServlet { - protected override def handleKill(submissionId: String): KillSubmissionResponse = { - val k = scheduler.killDriver(submissionId) - k.serverSparkVersion = sparkVersion - k - } -} - -private[mesos] class MesosStatusRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) - extends StatusRequestServlet { - protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { - val d = scheduler.getDriverStatus(submissionId) - d.serverSparkVersion = sparkVersion - d - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala deleted file mode 100644 index 366f6d0bf392c..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.executor - -import java.nio.ByteBuffer - -import scala.jdk.CollectionConverters._ - -import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver} -import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} -import org.apache.mesos.protobuf.ByteString - -import org.apache.spark.{SparkConf, SparkEnv, TaskState} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.EXECUTOR_ID -import org.apache.spark.resource.ResourceInformation -import org.apache.spark.scheduler.TaskDescription -import org.apache.spark.scheduler.cluster.mesos.MesosSchedulerBackendUtil -import org.apache.spark.util.Utils - -private[spark] class MesosExecutorBackend - extends MesosExecutor - with ExecutorBackend - with Logging { - - var executor: Executor = null - var driver: ExecutorDriver = null - - override def statusUpdate(taskId: Long, state: TaskState.TaskState, data: ByteBuffer): Unit = { - val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() - driver.sendStatusUpdate(MesosTaskStatus.newBuilder() - .setTaskId(mesosTaskId) - .setState(MesosSchedulerBackendUtil.taskStateToMesos(state)) - .setData(ByteString.copyFrom(data)) - .build()) - } - - override def registered( - driver: ExecutorDriver, - executorInfo: ExecutorInfo, - frameworkInfo: FrameworkInfo, - agentInfo: SlaveInfo): Unit = { - - // Get num cores for this task from ExecutorInfo, created in MesosSchedulerBackend. - val cpusPerTask = executorInfo.getResourcesList.asScala - .find(_.getName == "cpus") - .map(_.getScalar.getValue.toInt) - .getOrElse(0) - val executorId = executorInfo.getExecutorId.getValue - - logInfo(s"Registered with Mesos as executor ID $executorId with $cpusPerTask cpus") - this.driver = driver - // Set a context class loader to be picked up by the serializer. Without this call - // the serializer would default to the null class loader, and fail to find Spark classes - // See SPARK-10986. - Thread.currentThread().setContextClassLoader(this.getClass.getClassLoader) - - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++ - Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue)) - val conf = new SparkConf(loadDefaults = true).setAll(properties) - conf.set(EXECUTOR_ID, executorId) - val env = SparkEnv.createExecutorEnv( - conf, executorId, agentInfo.getHostname, cpusPerTask, None, isLocal = false) - - executor = new Executor( - executorId, - agentInfo.getHostname, - env, - resources = Map.empty[String, ResourceInformation]) - } - - override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo): Unit = { - val taskDescription = TaskDescription.decode(taskInfo.getData.asReadOnlyByteBuffer()) - if (executor == null) { - logError("Received launchTask but executor was null") - } else { - SparkHadoopUtil.get.runAsSparkUser { () => - executor.launchTask(this, taskDescription) - } - } - } - - override def error(d: ExecutorDriver, message: String): Unit = { - logError("Error from Mesos: " + message) - } - - override def killTask(d: ExecutorDriver, t: TaskID): Unit = { - if (executor == null) { - logError("Received KillTask but executor was null") - } else { - // TODO: Determine the 'interruptOnCancel' property set for the given job. - executor.killTask( - t.getValue.toLong, interruptThread = false, reason = "killed by mesos") - } - } - - override def reregistered(d: ExecutorDriver, p2: SlaveInfo): Unit = {} - - override def disconnected(d: ExecutorDriver): Unit = {} - - override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]): Unit = {} - - override def shutdown(d: ExecutorDriver): Unit = {} -} - -/** - * Entry point for Mesos executor. - */ -private[spark] object MesosExecutorBackend extends Logging { - def main(args: Array[String]): Unit = { - Utils.initDaemon(log) - // Create a new Executor and start it running - val runner = new MesosExecutorBackend() - new MesosExecutorDriver(runner).run() - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala deleted file mode 100644 index ed1b3d7a16739..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import org.apache.spark.SparkContext -import org.apache.spark.deploy.mesos.config._ -import org.apache.spark.internal.config._ -import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} - -/** - * Cluster Manager for creation of Mesos scheduler and backend - */ -private[spark] class MesosClusterManager extends ExternalClusterManager { - private val MESOS_REGEX = """mesos://(.*)""".r - - override def canCreate(masterURL: String): Boolean = { - masterURL.startsWith("mesos") - } - - override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { - new TaskSchedulerImpl(sc) - } - - override def createSchedulerBackend(sc: SparkContext, - masterURL: String, - scheduler: TaskScheduler): SchedulerBackend = { - require(!sc.conf.get(IO_ENCRYPTION_ENABLED), - "I/O encryption is currently not supported in Mesos.") - - val mesosUrl = MESOS_REGEX.findFirstMatchIn(masterURL).get.group(1) - val coarse = sc.conf.get(COARSE_MODE) - if (coarse) { - new MesosCoarseGrainedSchedulerBackend( - scheduler.asInstanceOf[TaskSchedulerImpl], - sc, - mesosUrl, - sc.env.securityManager) - } else { - new MesosFineGrainedSchedulerBackend( - scheduler.asInstanceOf[TaskSchedulerImpl], - sc, - mesosUrl) - } - } - - override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { - scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) - } -} - diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala deleted file mode 100644 index fa5977c8f8440..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import scala.jdk.CollectionConverters._ - -import org.apache.curator.framework.CuratorFramework -import org.apache.zookeeper.CreateMode -import org.apache.zookeeper.KeeperException.NoNodeException - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkCuratorUtil -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.Deploy._ -import org.apache.spark.util.Utils - -/** - * Persistence engine factory that is responsible for creating new persistence engines - * to store Mesos cluster mode state. - */ -private[spark] abstract class MesosClusterPersistenceEngineFactory(conf: SparkConf) { - def createEngine(path: String): MesosClusterPersistenceEngine -} - -/** - * Mesos cluster persistence engine is responsible for persisting Mesos cluster mode - * specific state, so that on failover all the state can be recovered and the scheduler - * can resume managing the drivers. - */ -private[spark] trait MesosClusterPersistenceEngine { - def persist(name: String, obj: Object): Unit - def expunge(name: String): Unit - def fetch[T](name: String): Option[T] - def fetchAll[T](): Iterable[T] -} - -/** - * Zookeeper backed persistence engine factory. - * All Zk engines created from this factory shares the same Zookeeper client, so - * all of them reuses the same connection pool. - */ -private[spark] class ZookeeperMesosClusterPersistenceEngineFactory(conf: SparkConf) - extends MesosClusterPersistenceEngineFactory(conf) with Logging { - - lazy val zk = SparkCuratorUtil.newClient(conf) - - def createEngine(path: String): MesosClusterPersistenceEngine = { - new ZookeeperMesosClusterPersistenceEngine(path, zk, conf) - } -} - -/** - * Black hole persistence engine factory that creates black hole - * persistence engines, which stores nothing. - */ -private[spark] class BlackHoleMesosClusterPersistenceEngineFactory - extends MesosClusterPersistenceEngineFactory(null) { - def createEngine(path: String): MesosClusterPersistenceEngine = { - new BlackHoleMesosClusterPersistenceEngine - } -} - -/** - * Black hole persistence engine that stores nothing. - */ -private[spark] class BlackHoleMesosClusterPersistenceEngine extends MesosClusterPersistenceEngine { - override def persist(name: String, obj: Object): Unit = {} - override def fetch[T](name: String): Option[T] = None - override def expunge(name: String): Unit = {} - override def fetchAll[T](): Iterable[T] = Iterable.empty[T] -} - -/** - * Zookeeper based Mesos cluster persistence engine, that stores cluster mode state - * into Zookeeper. Each engine object is operating under one folder in Zookeeper, but - * reuses a shared Zookeeper client. - */ -private[spark] class ZookeeperMesosClusterPersistenceEngine( - baseDir: String, - zk: CuratorFramework, - conf: SparkConf) - extends MesosClusterPersistenceEngine with Logging { - private val workingDir = - conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark_mesos_dispatcher") + "/" + baseDir - - SparkCuratorUtil.mkdir(zk, workingDir) - - def path(name: String): String = { - workingDir + "/" + name - } - - override def expunge(name: String): Unit = { - zk.delete().forPath(path(name)) - } - - override def persist(name: String, obj: Object): Unit = { - val serialized = Utils.serialize(obj) - val zkPath = path(name) - zk.create().withMode(CreateMode.PERSISTENT).forPath(zkPath, serialized) - } - - override def fetch[T](name: String): Option[T] = { - val zkPath = path(name) - - try { - val fileData = zk.getData().forPath(zkPath) - Some(Utils.deserialize[T](fileData)) - } catch { - case e: NoNodeException => None - case e: Exception => - logWarning("Exception while reading persisted file, deleting", e) - zk.delete().forPath(zkPath) - None - } - } - - override def fetchAll[T](): Iterable[T] = { - zk.getChildren.forPath(workingDir).asScala.flatMap(fetch[T]) - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala deleted file mode 100644 index aed08d13e445e..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ /dev/null @@ -1,904 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.io.File -import java.util.{Collections, Date, List => JList} - -import scala.collection.immutable -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer -import scala.jdk.CollectionConverters._ - -import org.apache.mesos.{Scheduler, SchedulerDriver} -import org.apache.mesos.Protos.{SlaveID => AgentID, TaskState => MesosTaskState, _} -import org.apache.mesos.Protos.Environment.Variable -import org.apache.mesos.Protos.TaskStatus.Reason - -import org.apache.spark.{SparkConf, SparkException, TaskState} -import org.apache.spark.deploy.mesos.{config, MesosDriverDescription} -import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} -import org.apache.spark.internal.config._ -import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} -import org.apache.spark.util.Utils - -/** - * Tracks the current state of a Mesos Task that runs a Spark driver. - * @param driverDescription Submitted driver description from - * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]] - * @param taskId Mesos TaskID generated for the task - * @param agentId Agent ID that the task is assigned to - * @param mesosTaskStatus The last known task status update. - * @param startDate The date the task was launched - * @param finishDate The date the task finished - * @param frameworkId Mesos framework ID the task registers with - */ -private[spark] class MesosClusterSubmissionState( - val driverDescription: MesosDriverDescription, - val taskId: TaskID, - val agentId: AgentID, - var mesosTaskStatus: Option[TaskStatus], - var startDate: Date, - var finishDate: Option[Date], - val frameworkId: String) - extends Serializable { - - def copy(): MesosClusterSubmissionState = { - new MesosClusterSubmissionState( - driverDescription, taskId, agentId, mesosTaskStatus, startDate, finishDate, frameworkId) - } -} - -/** - * Tracks the retry state of a driver, which includes the next time it should be scheduled - * and necessary information to do exponential backoff. - * This class is not thread-safe, and we expect the caller to handle synchronizing state. - * - * @param lastFailureStatus Last Task status when it failed. - * @param retries Number of times it has been retried. - * @param nextRetry Time at which it should be retried next - * @param waitTime The amount of time driver is scheduled to wait until next retry. - */ -private[spark] class MesosClusterRetryState( - val lastFailureStatus: TaskStatus, - val retries: Int, - val nextRetry: Date, - val waitTime: Int) extends Serializable { - def copy(): MesosClusterRetryState = - new MesosClusterRetryState(lastFailureStatus, retries, nextRetry, waitTime) -} - -/** - * The full state of the cluster scheduler, currently being used for displaying - * information on the UI. - * - * @param frameworkId Mesos Framework id for the cluster scheduler. - * @param masterUrl The Mesos master url - * @param queuedDrivers All drivers queued to be launched - * @param launchedDrivers All launched or running drivers - * @param finishedDrivers All terminated drivers - * @param pendingRetryDrivers All drivers pending to be retried - */ -private[spark] class MesosClusterSchedulerState( - val frameworkId: String, - val masterUrl: Option[String], - val queuedDrivers: Iterable[MesosDriverDescription], - val launchedDrivers: Iterable[MesosClusterSubmissionState], - val finishedDrivers: Iterable[MesosClusterSubmissionState], - val pendingRetryDrivers: Iterable[MesosDriverDescription]) - -/** - * The full state of a Mesos driver, that is being used to display driver information on the UI. - */ -private[spark] class MesosDriverState( - val state: String, - val description: MesosDriverDescription, - val submissionState: Option[MesosClusterSubmissionState] = None) - -/** - * A Mesos scheduler that is responsible for launching submitted Spark drivers in cluster mode - * as Mesos tasks in a Mesos cluster. - * All drivers are launched asynchronously by the framework, which will eventually be launched - * by one of the agents in the cluster. The results of the driver will be stored in agent's task - * sandbox which is accessible by visiting the Mesos UI. - * This scheduler supports recovery by persisting all its state and performs task reconciliation - * on recover, which gets all the latest state for all the drivers from Mesos master. - */ -private[spark] class MesosClusterScheduler( - engineFactory: MesosClusterPersistenceEngineFactory, - conf: SparkConf) - extends Scheduler with MesosSchedulerUtils with MesosScheduler { - var frameworkUrl: String = _ - private val metricsSystem = - MetricsSystem.createMetricsSystem(MetricsSystemInstances.MESOS_CLUSTER, conf) - private val master = conf.get("spark.master") - private val appName = conf.get("spark.app.name") - private val queuedCapacity = conf.get(config.MAX_DRIVERS) - private val retainedDrivers = conf.get(config.RETAINED_DRIVERS) - private val maxRetryWaitTime = conf.get(config.CLUSTER_RETRY_WAIT_MAX_SECONDS) - private val queues: immutable.Map[String, Float] = - conf.getAllWithPrefix("spark.mesos.dispatcher.queue.").map(t => (t._1, t._2.toFloat)).toMap - private val schedulerState = engineFactory.createEngine("scheduler") - private val stateLock = new Object() - // Keyed by submission id - private val finishedDrivers = - new mutable.ArrayBuffer[MesosClusterSubmissionState](retainedDrivers) - private var frameworkId: String = null - // Holds all the launched drivers and current launch state, keyed by submission id. - private val launchedDrivers = new mutable.HashMap[String, MesosClusterSubmissionState]() - // Holds a map of driver id to expected agent id that is passed to Mesos for reconciliation. - // All drivers that are loaded after failover are added here, as we need get the latest - // state of the tasks from Mesos. Keyed by task Id. - private val pendingRecover = new mutable.HashMap[String, AgentID]() - // Stores all the submitted drivers that hasn't been launched, keyed by submission id - // and sorted by priority, then by submission date - private val driverOrdering = new Ordering[MesosDriverDescription] { - override def compare(x: MesosDriverDescription, y: MesosDriverDescription): Int = { - val xp = getDriverPriority(x) - val yp = getDriverPriority(y) - if (xp != yp) { - xp compare yp - } else { - y.submissionDate.compareTo(x.submissionDate) - } - } - } - private val queuedDrivers = new mutable.TreeSet[MesosDriverDescription]()(driverOrdering.reverse) - // All supervised drivers that are waiting to retry after termination, keyed by submission id - private val pendingRetryDrivers = new ArrayBuffer[MesosDriverDescription]() - private val queuedDriversState = engineFactory.createEngine("driverQueue") - private val launchedDriversState = engineFactory.createEngine("launchedDrivers") - private val pendingRetryDriversState = engineFactory.createEngine("retryList") - private final val RETRY_SEP = "-retry-" - // Flag to mark if the scheduler is ready to be called, which is until the scheduler - // is registered with Mesos master. - @volatile protected var ready = false - private var masterInfo: Option[MasterInfo] = None - private var schedulerDriver: SchedulerDriver = _ - - def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = { - val c = new CreateSubmissionResponse - if (!ready) { - c.success = false - c.message = "Scheduler is not ready to take requests" - return c - } - - stateLock.synchronized { - if (isQueueFull()) { - c.success = false - c.message = "Already reached maximum submission size" - return c - } - c.submissionId = desc.submissionId - c.success = true - addDriverToQueue(desc) - } - c - } - - def killDriver(submissionId: String): KillSubmissionResponse = { - val k = new KillSubmissionResponse - if (!ready) { - k.success = false - k.message = "Scheduler is not ready to take requests" - return k - } - k.submissionId = submissionId - stateLock.synchronized { - // We look for the requested driver in the following places: - // 1. Check if submission is running or launched. - // 2. Check if it's still queued. - // 3. Check if it's in the retry list. - // 4. Check if it has already completed. - if (launchedDrivers.contains(submissionId)) { - val state = launchedDrivers(submissionId) - schedulerDriver.killTask(state.taskId) - k.success = true - k.message = "Killing running driver" - } else if (removeFromQueuedDrivers(submissionId)) { - k.success = true - k.message = "Removed driver while it's still pending" - } else if (removeFromPendingRetryDrivers(submissionId)) { - k.success = true - k.message = "Removed driver while it's being retried" - } else if (finishedDrivers.exists(_.driverDescription.submissionId == submissionId)) { - k.success = false - k.message = "Driver already terminated" - } else { - k.success = false - k.message = "Cannot find driver" - } - } - k - } - - def getDriverStatus(submissionId: String): SubmissionStatusResponse = { - val s = new SubmissionStatusResponse - if (!ready) { - s.success = false - s.message = "Scheduler is not ready to take requests" - return s - } - s.submissionId = submissionId - stateLock.synchronized { - if (queuedDrivers.exists(_.submissionId == submissionId)) { - s.success = true - s.driverState = "QUEUED" - } else if (launchedDrivers.contains(submissionId)) { - s.success = true - s.driverState = "RUNNING" - launchedDrivers(submissionId).mesosTaskStatus.foreach(state => s.message = state.toString) - } else if (finishedDrivers.exists(_.driverDescription.submissionId == submissionId)) { - s.success = true - s.driverState = "FINISHED" - finishedDrivers - .find(d => d.driverDescription.submissionId.equals(submissionId)).get.mesosTaskStatus - .foreach(state => s.message = state.toString) - } else if (pendingRetryDrivers.exists(_.submissionId == submissionId)) { - val status = pendingRetryDrivers.find(_.submissionId == submissionId) - .get.retryState.get.lastFailureStatus - s.success = true - s.driverState = "RETRYING" - s.message = status.toString - } else { - s.success = false - s.driverState = "NOT_FOUND" - } - } - s - } - - /** - * Gets the driver state to be displayed on the Web UI. - */ - def getDriverState(submissionId: String): Option[MesosDriverState] = { - stateLock.synchronized { - queuedDrivers.find(_.submissionId == submissionId) - .map(d => new MesosDriverState("QUEUED", d)) - .orElse(launchedDrivers.get(submissionId) - .map(d => new MesosDriverState("RUNNING", d.driverDescription, Some(d)))) - .orElse(finishedDrivers.find(_.driverDescription.submissionId == submissionId) - .map(d => new MesosDriverState("FINISHED", d.driverDescription, Some(d)))) - .orElse(pendingRetryDrivers.find(_.submissionId == submissionId) - .map(d => new MesosDriverState("RETRYING", d))) - } - } - - private def isQueueFull(): Boolean = launchedDrivers.size >= queuedCapacity - - /** - * Recover scheduler state that is persisted. - * We still need to do task reconciliation to be up to date of the latest task states - * as it might have changed while the scheduler is failing over. - */ - private def recoverState(): Unit = { - stateLock.synchronized { - launchedDriversState.fetchAll[MesosClusterSubmissionState]().foreach { state => - launchedDrivers(state.driverDescription.submissionId) = state - pendingRecover(state.taskId.getValue) = state.agentId - } - queuedDriversState.fetchAll[MesosDriverDescription]().foreach(d => queuedDrivers += d) - // There is potential timing issue where a queued driver might have been launched - // but the scheduler shuts down before the queued driver was able to be removed - // from the queue. We try to mitigate this issue by walking through all queued drivers - // and remove if they're already launched. - queuedDrivers - .filter(d => launchedDrivers.contains(d.submissionId)) - .foreach(d => removeFromQueuedDrivers(d.submissionId)) - pendingRetryDriversState.fetchAll[MesosDriverDescription]() - .foreach(s => pendingRetryDrivers += s) - // TODO: Consider storing finished drivers so we can show them on the UI after - // failover. For now we clear the history on each recovery. - finishedDrivers.clear() - } - } - - /** - * Starts the cluster scheduler and wait until the scheduler is registered. - * This also marks the scheduler to be ready for requests. - */ - def start(): Unit = { - // TODO: Implement leader election to make sure only one framework running in the cluster. - val fwId = schedulerState.fetch[String]("frameworkId") - fwId.foreach { id => - frameworkId = id - } - recoverState() - metricsSystem.registerSource(new MesosClusterSchedulerSource(this)) - metricsSystem.start() - val driver = createSchedulerDriver( - master, - MesosClusterScheduler.this, - Utils.getCurrentUserName(), - appName, - conf, - Some(frameworkUrl), - Some(true), - Some(Integer.MAX_VALUE), - fwId) - - startScheduler(driver) - ready = true - } - - def stop(): Unit = { - ready = false - metricsSystem.report() - metricsSystem.stop() - schedulerDriver.stop(true) - } - - override def registered( - driver: SchedulerDriver, - newFrameworkId: FrameworkID, - masterInfo: MasterInfo): Unit = { - logInfo("Registered as framework ID " + newFrameworkId.getValue) - if (newFrameworkId.getValue != frameworkId) { - frameworkId = newFrameworkId.getValue - schedulerState.persist("frameworkId", frameworkId) - } - markRegistered() - - stateLock.synchronized { - this.masterInfo = Some(masterInfo) - this.schedulerDriver = driver - - if (!pendingRecover.isEmpty) { - // Start task reconciliation if we need to recover. - val statuses = pendingRecover.collect { - case (taskId, agentId) => - val newStatus = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(taskId).build()) - .setSlaveId(agentId) - .setState(MesosTaskState.TASK_STAGING) - .build() - launchedDrivers.get(getSubmissionIdFromTaskId(taskId)) - .map(_.mesosTaskStatus.getOrElse(newStatus)) - .getOrElse(newStatus) - } - // TODO: Page the status updates to avoid trying to reconcile - // a large amount of tasks at once. - driver.reconcileTasks(statuses.toSeq.asJava) - } - } - } - - private def getDriverExecutorURI(desc: MesosDriverDescription): Option[String] = { - desc.conf.get(config.EXECUTOR_URI).orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) - } - - private def getDriverFrameworkID(desc: MesosDriverDescription): String = { - val retries = desc.retryState.map { d => s"${RETRY_SEP}${d.retries.toString}" }.getOrElse("") - s"${frameworkId}-${desc.submissionId}${retries}" - } - - private[mesos] def getDriverPriority(desc: MesosDriverDescription): Float = { - val defaultQueueName = config.DISPATCHER_QUEUE.defaultValueString - val queueName = desc.conf.get("spark.mesos.dispatcher.queue", defaultQueueName) - if (queueName != defaultQueueName) { - queues.getOrElse(queueName, throw new NoSuchElementException(queueName)) - } else { - 0.0f - } - } - - private def getDriverTaskId(desc: MesosDriverDescription): String = { - val sId = desc.submissionId - desc.retryState.map(state => sId + s"${RETRY_SEP}${state.retries.toString}").getOrElse(sId) - } - - private def getSubmissionIdFromTaskId(taskId: String): String = { - taskId.split(s"${RETRY_SEP}").head - } - - private def adjust[A, B](m: Map[A, B], k: A, default: B)(f: B => B) = { - m.updated(k, f(m.getOrElse(k, default))) - } - - private def getDriverEnvironment(desc: MesosDriverDescription): Environment = { - // TODO(mgummelt): Don't do this here. This should be passed as a --conf - val commandEnv = adjust(desc.command.environment.toMap, "SPARK_SUBMIT_OPTS", "")( - v => s"$v -D${config.DRIVER_FRAMEWORK_ID.key}=${getDriverFrameworkID(desc)}" - ) - - val env = desc.conf.getAllWithPrefix(config.DRIVER_ENV_PREFIX) ++ commandEnv - - val envBuilder = Environment.newBuilder() - - // add normal environment variables - env.foreach { case (k, v) => - envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v)) - } - - // add secret environment variables - MesosSchedulerBackendUtil.getSecretEnvVar(desc.conf, config.driverSecretConfig) - .foreach { variable => - if (variable.getSecret.getReference.isInitialized) { - logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName} " + - s"on file ${variable.getName}") - } else { - logInfo(s"Setting secret on environment variable name=${variable.getName}") - } - envBuilder.addVariables(variable) - } - - envBuilder.build() - } - - private def isContainerLocalAppJar(desc: MesosDriverDescription): Boolean = { - val isLocalJar = desc.jarUrl.startsWith("local://") - val isContainerLocal = desc.conf.get(config.APP_JAR_LOCAL_RESOLUTION_MODE) match { - case "container" => true - case "host" => false - } - isLocalJar && isContainerLocal - } - - private def getDriverUris(desc: MesosDriverDescription): List[CommandInfo.URI] = { - val useFetchCache = desc.conf.get(config.ENABLE_FETCHER_CACHE) || - conf.get(config.ENABLE_FETCHER_CACHE) - val confUris = (conf.get(config.URIS_TO_DOWNLOAD) ++ - desc.conf.get(config.URIS_TO_DOWNLOAD) ++ - desc.conf.get(SUBMIT_PYTHON_FILES)).toList - - if (isContainerLocalAppJar(desc)) { - (confUris ++ getDriverExecutorURI(desc).toList).map(uri => - CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetchCache).build()) - } else { - val jarUrl = desc.jarUrl.stripPrefix("file:").stripPrefix("local:") - ((jarUrl :: confUris) ++ getDriverExecutorURI(desc).toList).map(uri => - CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetchCache).build()) - } - } - - private def getContainerInfo(desc: MesosDriverDescription): ContainerInfo.Builder = { - val containerInfo = MesosSchedulerBackendUtil.buildContainerInfo(desc.conf) - - MesosSchedulerBackendUtil.getSecretVolume(desc.conf, config.driverSecretConfig) - .foreach { volume => - if (volume.getSource.getSecret.getReference.isInitialized) { - logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName} " + - s"on file ${volume.getContainerPath}") - } else { - logInfo(s"Setting secret on file name=${volume.getContainerPath}") - } - containerInfo.addVolumes(volume) - } - - containerInfo - } - - private[mesos] def getDriverCommandValue(desc: MesosDriverDescription): String = { - val dockerDefined = desc.conf.contains(config.EXECUTOR_DOCKER_IMAGE) - val executorUri = getDriverExecutorURI(desc) - // Gets the path to run spark-submit, and the path to the Mesos sandbox. - val (executable, sandboxPath) = if (dockerDefined) { - // Application jar is automatically downloaded in the mounted sandbox by Mesos, - // and the path to the mounted volume is stored in $MESOS_SANDBOX env variable. - ("./bin/spark-submit", "$MESOS_SANDBOX") - } else if (executorUri.isDefined) { - val folderBasename = executorUri.get.split('/').last.split('.').head - - val entries = conf.get(EXECUTOR_LIBRARY_PATH) - .map(path => Seq(path) ++ desc.command.libraryPathEntries) - .getOrElse(desc.command.libraryPathEntries) - - val prefixEnv = if (!entries.isEmpty) Utils.libraryPathEnvPrefix(entries) else "" - - val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit" - // Sandbox path points to the parent folder as we chdir into the folderBasename. - (cmdExecutable, "..") - } else { - val executorSparkHome = desc.conf.get(config.EXECUTOR_HOME) - .orElse(conf.getOption("spark.home")) - .orElse(Option(System.getenv("SPARK_HOME"))) - .getOrElse { - throw new SparkException(s"Executor Spark home `${config.EXECUTOR_HOME}` is not set!") - } - val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getPath - // Sandbox points to the current directory by default with Mesos. - (cmdExecutable, ".") - } - val cmdOptions = generateCmdOption(desc, sandboxPath).mkString(" ") - val primaryResource = { - if (isContainerLocalAppJar(desc)) { - new File(desc.jarUrl.stripPrefix("local://")).toString() - } else { - new File(sandboxPath, desc.jarUrl.split("/").last).toString() - } - } - - val appArguments = desc.command.arguments.map(shellEscape).mkString(" ") - - s"$executable $cmdOptions $primaryResource $appArguments" - } - - private def buildDriverCommand(desc: MesosDriverDescription): CommandInfo = { - val builder = CommandInfo.newBuilder() - builder.setValue(getDriverCommandValue(desc)) - builder.setEnvironment(getDriverEnvironment(desc)) - builder.addAllUris(getDriverUris(desc).asJava) - builder.build() - } - - private def generateCmdOption(desc: MesosDriverDescription, sandboxPath: String): Seq[String] = { - var options = Seq( - "--name", desc.conf.get("spark.app.name"), - "--master", s"mesos://${conf.get("spark.master")}", - "--driver-cores", desc.cores.toString, - "--driver-memory", s"${desc.mem}M") - - // Assume empty main class means we're running python - if (!desc.command.mainClass.equals("")) { - options ++= Seq("--class", desc.command.mainClass) - } - - desc.conf.getOption(EXECUTOR_MEMORY.key).foreach { v => - options ++= Seq("--executor-memory", v) - } - desc.conf.getOption(CORES_MAX.key).foreach { v => - options ++= Seq("--total-executor-cores", v) - } - - val pyFiles = desc.conf.get(SUBMIT_PYTHON_FILES) - val formattedFiles = pyFiles.map { path => - new File(sandboxPath, path.split("/").last).toString() - }.mkString(",") - if (!formattedFiles.equals("")) { - options ++= Seq("--py-files", formattedFiles) - } - - // --conf - val replicatedOptionsExcludeList = Set( - JARS.key, // Avoids duplicate classes in classpath - SUBMIT_DEPLOY_MODE.key, // this would be set to `cluster`, but we need client - "spark.master" // this contains the address of the dispatcher, not master - ) - val defaultConf = conf.getAllWithPrefix(config.DISPATCHER_DRIVER_DEFAULT_PREFIX).toMap - val driverConf = desc.conf.getAll - .filter { case (key, _) => !replicatedOptionsExcludeList.contains(key) } - .toMap - (defaultConf ++ driverConf).toSeq.sortBy(_._1).foreach { case (key, value) => - options ++= Seq("--conf", s"${key}=${value}") } - - options.map(shellEscape) - } - - /** - * Escape args for Unix-like shells, unless already quoted by the user. - * Based on: http://www.gnu.org/software/bash/manual/html_node/Double-Quotes.html - * and http://www.grymoire.com/Unix/Quote.html - * - * @param value argument - * @return escaped argument - */ - private[scheduler] def shellEscape(value: String): String = { - val WrappedInQuotes = """^(".+"|'.+')$""".r - val ShellSpecialChars = (""".*([ '<>&|\?\*;!#\\(\)"$`]).*""").r - value match { - case WrappedInQuotes(c) => value // The user quoted his args, don't touch it! - case ShellSpecialChars(c) => "\"" + value.replaceAll("""(["`\$\\])""", """\\$1""") + "\"" - case _: String => value // Don't touch harmless strings - } - } - - private class ResourceOffer( - val offer: Offer, - var remainingResources: JList[Resource], - var attributes: JList[Attribute]) { - override def toString(): String = { - s"Offer id: ${offer.getId}, resources: ${remainingResources}, attributes: ${attributes}" - } - } - - private def createTaskInfo(desc: MesosDriverDescription, offer: ResourceOffer): TaskInfo = { - val taskId = TaskID.newBuilder().setValue(getDriverTaskId(desc)).build() - - val (remainingResources, cpuResourcesToUse) = - partitionResources(offer.remainingResources, "cpus", desc.cores) - val (finalResources, memResourcesToUse) = - partitionResources(remainingResources.asJava, "mem", driverContainerMemory(desc)) - offer.remainingResources = finalResources.asJava - - val appName = desc.conf.get("spark.app.name") - - val driverLabels = MesosProtoUtils.mesosLabels(desc.conf.get(config.DRIVER_LABELS) - .getOrElse("")) - - TaskInfo.newBuilder() - .setTaskId(taskId) - .setName(s"Driver for ${appName}") - .setSlaveId(offer.offer.getSlaveId) - .setCommand(buildDriverCommand(desc)) - .setContainer(getContainerInfo(desc)) - .addAllResources(cpuResourcesToUse.asJava) - .addAllResources(memResourcesToUse.asJava) - .setLabels(driverLabels) - .build - } - - /** - * This method takes all the possible candidates and attempt to schedule them with Mesos offers. - * Every time a new task is scheduled, the afterLaunchCallback is called to perform post scheduled - * logic on each task. - */ - private def scheduleTasks( - candidates: Seq[MesosDriverDescription], - afterLaunchCallback: (String) => Boolean, - currentOffers: List[ResourceOffer], - tasks: mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]): Unit = { - for (submission <- candidates) { - val driverCpu = submission.cores - val driverMem = driverContainerMemory(submission) - val driverConstraints = - parseConstraintString(submission.conf.get(config.DRIVER_CONSTRAINTS)) - logTrace(s"Finding offer to launch driver with cpu: $driverCpu, mem: $driverMem, " + - s"driverConstraints: $driverConstraints") - val offerOption = currentOffers.find { offer => - getResource(offer.remainingResources, "cpus") >= driverCpu && - getResource(offer.remainingResources, "mem") >= driverMem && - matchesAttributeRequirements(driverConstraints, toAttributeMap(offer.attributes)) - } - if (offerOption.isEmpty) { - logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId}, " + - s"cpu: $driverCpu, mem: $driverMem") - } else { - val offer = offerOption.get - val queuedTasks = tasks.getOrElseUpdate(offer.offer.getId, new ArrayBuffer[TaskInfo]) - try { - val task = createTaskInfo(submission, offer) - queuedTasks += task - logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + - submission.submissionId + s" with taskId: ${task.getTaskId.toString}") - val newState = new MesosClusterSubmissionState( - submission, - task.getTaskId, - offer.offer.getSlaveId, - None, - new Date(), - None, - getDriverFrameworkID(submission)) - launchedDrivers(submission.submissionId) = newState - launchedDriversState.persist(submission.submissionId, newState) - afterLaunchCallback(submission.submissionId) - } catch { - case e: SparkException => - afterLaunchCallback(submission.submissionId) - finishedDrivers += new MesosClusterSubmissionState( - submission, - TaskID.newBuilder().setValue(submission.submissionId).build(), - AgentID.newBuilder().setValue("").build(), - None, - null, - None, - getDriverFrameworkID(submission)) - logError(s"Failed to launch the driver with id: ${submission.submissionId}, " + - s"cpu: $driverCpu, mem: $driverMem, reason: ${e.getMessage}") - } - } - } - } - - override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { - logTrace(s"Received offers from Mesos: \n${offers.asScala.mkString("\n")}") - val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() - val currentTime = new Date() - - val currentOffers = offers.asScala.map { - offer => new ResourceOffer(offer, offer.getResourcesList, offer.getAttributesList) - }.toList - - stateLock.synchronized { - // We first schedule all the supervised drivers that are ready to retry. - // This list will be empty if none of the drivers are marked as supervise. - val driversToRetry = pendingRetryDrivers.filter { d => - d.retryState.get.nextRetry.before(currentTime) - } - - scheduleTasks( - copyBuffer(driversToRetry).toSeq, - removeFromPendingRetryDrivers, - currentOffers, - tasks) - - // Then we walk through the queued drivers and try to schedule them. - scheduleTasks( - copyBuffer(queuedDrivers).toSeq, - removeFromQueuedDrivers, - currentOffers, - tasks) - } - tasks.foreach { case (offerId, taskInfos) => - driver.launchTasks(Collections.singleton(offerId), taskInfos.asJava) - } - - for (offer <- currentOffers if !tasks.contains(offer.offer.getId)) { - declineOffer(driver, offer.offer, None, Some(getRejectOfferDuration(conf))) - } - } - - private def copyBuffer( - buffer: TraversableOnce[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { - val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) - buffer.copyToBuffer(newBuffer) - newBuffer - } - - def getSchedulerState(): MesosClusterSchedulerState = { - stateLock.synchronized { - new MesosClusterSchedulerState( - frameworkId, - masterInfo.map(m => s"http://${m.getIp}:${m.getPort}"), - copyBuffer(queuedDrivers), - launchedDrivers.values.map(_.copy()).toList, - finishedDrivers.map(_.copy()).toList, - copyBuffer(pendingRetryDrivers)) - } - } - - override def offerRescinded(driver: SchedulerDriver, offerId: OfferID): Unit = {} - override def disconnected(driver: SchedulerDriver): Unit = {} - override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = { - logInfo(s"Framework re-registered with master ${masterInfo.getId}") - } - override def agentLost(driver: SchedulerDriver, agentId: AgentID): Unit = {} - override def error(driver: SchedulerDriver, error: String): Unit = { - logError("Error received: " + error) - markErr() - } - - /** - * Check if the task state is a recoverable state that we can relaunch the task. - * Task state like TASK_ERROR are not relaunchable state since it wasn't able - * to be validated by Mesos. - */ - private def shouldRelaunch(state: MesosTaskState): Boolean = { - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_LOST - } - - override def statusUpdate(driver: SchedulerDriver, status: TaskStatus): Unit = { - val taskId = status.getTaskId.getValue - - logInfo(s"Received status update: taskId=${taskId}" + - s" state=${status.getState}" + - s" message=${status.getMessage}" + - s" reason=${status.getReason}") - - stateLock.synchronized { - val subId = getSubmissionIdFromTaskId(taskId) - if (launchedDrivers.contains(subId)) { - if (status.getReason == Reason.REASON_RECONCILIATION && - !pendingRecover.contains(taskId)) { - // Task has already received update and no longer requires reconciliation. - return - } - val state = launchedDrivers(subId) - // Check if the driver is supervise enabled and can be relaunched. - if (state.driverDescription.supervise && shouldRelaunch(status.getState)) { - if (isTaskOutdated(taskId, state)) { - // Prevent outdated task from overwriting a more recent status - return - } - removeFromLaunchedDrivers(subId) - state.finishDate = Some(new Date()) - val retryState: Option[MesosClusterRetryState] = state.driverDescription.retryState - val (retries, waitTimeSec) = retryState - .map { rs => (rs.retries + 1, Math.min(maxRetryWaitTime, rs.waitTime * 2)) } - .getOrElse{ (1, 1) } - val nextRetry = new Date(new Date().getTime + waitTimeSec * 1000L) - val newDriverDescription = state.driverDescription.copy( - retryState = Some(new MesosClusterRetryState(status, retries, nextRetry, waitTimeSec))) - addDriverToPending(newDriverDescription, newDriverDescription.submissionId) - } else if (TaskState.isFinished(mesosToTaskState(status.getState))) { - retireDriver(subId, state) - } - state.mesosTaskStatus = Option(status) - } else { - logError(s"Unable to find driver with $taskId in status update") - } - } - } - - /** - * Check if the task is outdated i.e. has already been launched or is pending - * If neither, the taskId is outdated and should be ignored - * This is to avoid scenarios where an outdated status update arrives - * after a supervised driver has already been relaunched - */ - private def isTaskOutdated(taskId: String, state: MesosClusterSubmissionState): Boolean = - taskId != state.taskId.getValue && - !pendingRetryDrivers.exists(_.submissionId == state.driverDescription.submissionId) - - private def retireDriver( - submissionId: String, - state: MesosClusterSubmissionState) = { - removeFromLaunchedDrivers(submissionId) - state.finishDate = Some(new Date()) - if (finishedDrivers.size >= retainedDrivers) { - val toRemove = math.max(retainedDrivers / 10, 1) - finishedDrivers.trimStart(toRemove) - } - finishedDrivers += state - } - - override def frameworkMessage( - driver: SchedulerDriver, - executorId: ExecutorID, - agentId: AgentID, - message: Array[Byte]): Unit = {} - - override def executorLost( - driver: SchedulerDriver, - executorId: ExecutorID, - agentId: AgentID, - status: Int): Unit = {} - - private def removeFromQueuedDrivers(subId: String): Boolean = { - val matchOption = queuedDrivers.find(_.submissionId == subId) - if (matchOption.isEmpty) { - false - } else { - queuedDrivers.remove(matchOption.get) - queuedDriversState.expunge(subId) - true - } - } - - private def removeFromLaunchedDrivers(subId: String): Boolean = { - if (launchedDrivers.remove(subId).isDefined) { - launchedDriversState.expunge(subId) - true - } else { - false - } - } - - private def removeFromPendingRetryDrivers(subId: String): Boolean = { - val index = pendingRetryDrivers.indexWhere(_.submissionId == subId) - if (index != -1) { - pendingRetryDrivers.remove(index) - pendingRetryDriversState.expunge(subId) - true - } else { - false - } - } - - def getQueuedDriversSize: Int = queuedDrivers.size - def getLaunchedDriversSize: Int = launchedDrivers.size - def getPendingRetryDriversSize: Int = pendingRetryDrivers.size - - private def addDriverToQueue(desc: MesosDriverDescription): Unit = { - queuedDriversState.persist(desc.submissionId, desc) - queuedDrivers += desc - revive() - } - - private def addDriverToPending(desc: MesosDriverDescription, subId: String) = { - pendingRetryDriversState.persist(subId, desc) - pendingRetryDrivers += desc - revive() - } - - private def revive(): Unit = { - logInfo("Reviving Offers.") - schedulerDriver.reviveOffers() - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala deleted file mode 100644 index 76aded4edb431..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import com.codahale.metrics.{Gauge, MetricRegistry} - -import org.apache.spark.metrics.source.Source - -private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterScheduler) - extends Source { - - override val sourceName: String = "mesos_cluster" - override val metricRegistry: MetricRegistry = new MetricRegistry() - - metricRegistry.register(MetricRegistry.name("waitingDrivers"), new Gauge[Int] { - override def getValue: Int = scheduler.getQueuedDriversSize - }) - - metricRegistry.register(MetricRegistry.name("launchedDrivers"), new Gauge[Int] { - override def getValue: Int = scheduler.getLaunchedDriversSize - }) - - metricRegistry.register(MetricRegistry.name("retryDrivers"), new Gauge[Int] { - override def getValue: Int = scheduler.getPendingRetryDriversSize - }) -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala deleted file mode 100644 index 29548de8b90eb..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ /dev/null @@ -1,820 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.io.File -import java.util.{Collections, List => JList} -import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} -import java.util.concurrent.locks.ReentrantLock - -import scala.collection.mutable -import scala.concurrent.Future -import scala.jdk.CollectionConverters._ - -import org.apache.mesos.Protos.{SlaveID => AgentID, TaskInfo => MesosTaskInfo, _} -import org.apache.mesos.SchedulerDriver - -import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException, TaskState} -import org.apache.spark.deploy.mesos.config._ -import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.internal.config -import org.apache.spark.internal.config.Network -import org.apache.spark.internal.config.Tests.IS_TESTING -import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} -import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.mesos.MesosExternalBlockStoreClient -import org.apache.spark.resource.ResourceProfile -import org.apache.spark.rpc.RpcEndpointAddress -import org.apache.spark.scheduler.{ExecutorProcessLost, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils - -/** - * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds - * onto each Mesos node for the duration of the Spark job instead of relinquishing cores whenever - * a task is done. It launches Spark tasks within the coarse-grained Mesos tasks using the - * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable - * latency. - * - * Unfortunately this has a bit of duplication from [[MesosFineGrainedSchedulerBackend]], - * but it seems hard to remove this. - */ -private[spark] class MesosCoarseGrainedSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext, - master: String, - securityManager: SecurityManager) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) - with MesosScheduler - with MesosSchedulerUtils { - - // Exclude an agent after this many failures - private val MAX_AGENT_FAILURES = 2 - - private val maxCoresOption = conf.get(config.CORES_MAX) - - private val executorCoresOption = conf.getOption(config.EXECUTOR_CORES.key).map(_.toInt) - - private val minCoresPerExecutor = executorCoresOption.getOrElse(1) - - // Maximum number of cores to acquire - private val maxCores = { - val cores = maxCoresOption.getOrElse(Int.MaxValue) - // Set maxCores to a multiple of smallest executor we can launch - cores - (cores % minCoresPerExecutor) - } - - private val useFetcherCache = conf.get(ENABLE_FETCHER_CACHE) - - private val maxGpus = conf.get(MAX_GPUS) - - private val taskLabels = conf.get(TASK_LABELS) - - private[this] val shutdownTimeoutMS = conf.get(COARSE_SHUTDOWN_TIMEOUT) - - // Synchronization protected by stateLock - private[this] var stopCalled: Boolean = false - - private val launcherBackend = new LauncherBackend() { - override protected def conf: SparkConf = sc.conf - - override protected def onStopRequest(): Unit = { - stopSchedulerBackend() - setState(SparkAppHandle.State.KILLED) - } - } - - // If shuffle service is enabled, the Spark driver will register with the shuffle service. - // This is for cleaning up shuffle files reliably. - private val shuffleServiceEnabled = conf.get(config.SHUFFLE_SERVICE_ENABLED) - - // Cores we have acquired with each Mesos task ID - private val coresByTaskId = new mutable.HashMap[String, Int] - private val gpusByTaskId = new mutable.HashMap[String, Int] - private var totalCoresAcquired = 0 - private var totalGpusAcquired = 0 - - // The amount of time to wait for locality scheduling - private val localityWaitNs = TimeUnit.MILLISECONDS.toNanos(conf.get(config.LOCALITY_WAIT)) - // The start of the waiting, for data local scheduling - private var localityWaitStartTimeNs = System.nanoTime() - // If true, the scheduler is in the process of launching executors to reach the requested - // executor limit - private var launchingExecutors = false - - // AgentID -> Agent - // This map accumulates entries for the duration of the job. Agents are never deleted, because - // we need to maintain e.g. failure state and connection state. - private val agents = new mutable.HashMap[String, Agent] - - /** - * The total number of executors we aim to have. Undefined when not using dynamic allocation. - * Initially set to 0 when using dynamic allocation, the executor allocation manager will send - * the real initial limit later. - */ - private var executorLimitOption: Option[Int] = { - if (Utils.isDynamicAllocationEnabled(conf)) { - Some(0) - } else { - None - } - } - - /** - * Return the current executor limit, which may be [[Int.MaxValue]] - * before properly initialized. - */ - private[mesos] def executorLimit: Int = executorLimitOption.getOrElse(Int.MaxValue) - - // private lock object protecting mutable state above. Using the intrinsic lock - // may lead to deadlocks since the superclass might also try to lock - private val stateLock = new ReentrantLock - - private val extraCoresPerExecutor = conf.get(EXTRA_CORES_PER_EXECUTOR) - - // Offer constraints - private val agentOfferConstraints = - parseConstraintString(sc.conf.get(CONSTRAINTS)) - - // Reject offers with mismatched constraints in seconds - private val rejectOfferDurationForUnmetConstraints = - getRejectOfferDurationForUnmetConstraints(sc.conf) - - // Reject offers when we reached the maximum number of cores for this framework - private val rejectOfferDurationForReachedMaxCores = - getRejectOfferDurationForReachedMaxCores(sc.conf) - - // A client for talking to the external shuffle service - private val mesosExternalShuffleClient: Option[MesosExternalBlockStoreClient] = { - if (shuffleServiceEnabled) { - Some(getShuffleClient()) - } else { - None - } - } - - // This method is factored out for testability - protected def getShuffleClient(): MesosExternalBlockStoreClient = { - new MesosExternalBlockStoreClient( - SparkTransportConf.fromSparkConf(conf, "shuffle"), - securityManager, - securityManager.isAuthenticationEnabled(), - conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT)) - } - - private var nextMesosTaskId = 0 - - @volatile var appId: String = _ - - private var schedulerDriver: SchedulerDriver = _ - - private val defaultProfile = sc.resourceProfileManager.defaultResourceProfile - - - def newMesosTaskId(): String = { - val id = nextMesosTaskId - nextMesosTaskId += 1 - id.toString - } - - override def start(): Unit = { - super.start() - - if (sc.deployMode == "client") { - launcherBackend.connect() - } - val startedBefore = IdHelper.startedBefore.getAndSet(true) - - val suffix = if (startedBefore) { - f"-${IdHelper.nextSCNumber.incrementAndGet()}%04d" - } else { - "" - } - - val driver = createSchedulerDriver( - master, - MesosCoarseGrainedSchedulerBackend.this, - sc.sparkUser, - sc.appName, - sc.conf, - sc.conf.get(DRIVER_WEBUI_URL).orElse(sc.ui.map(_.webUrl)), - None, - Some(sc.conf.get(DRIVER_FAILOVER_TIMEOUT)), - sc.conf.get(DRIVER_FRAMEWORK_ID).map(_ + suffix) - ) - - launcherBackend.setState(SparkAppHandle.State.SUBMITTED) - startScheduler(driver) - } - - def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = { - val environment = Environment.newBuilder() - val extraClassPath = conf.get(config.EXECUTOR_CLASS_PATH) - extraClassPath.foreach { cp => - environment.addVariables( - Environment.Variable.newBuilder().setName("SPARK_EXECUTOR_CLASSPATH").setValue(cp).build()) - } - val extraJavaOpts = conf.get(config.EXECUTOR_JAVA_OPTIONS).map { - Utils.substituteAppNExecIds(_, appId, taskId) - }.getOrElse("") - - // Set the environment variable through a command prefix - // to append to the existing value of the variable - val prefixEnv = conf.get(config.EXECUTOR_LIBRARY_PATH).map { p => - Utils.libraryPathEnvPrefix(Seq(p)) - }.getOrElse("") - - environment.addVariables( - Environment.Variable.newBuilder() - .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraJavaOpts) - .build()) - - sc.executorEnvs.foreach { case (key, value) => - environment.addVariables(Environment.Variable.newBuilder() - .setName(key) - .setValue(value) - .build()) - } - - MesosSchedulerBackendUtil.getSecretEnvVar(conf, executorSecretConfig).foreach { variable => - if (variable.getSecret.getReference.isInitialized) { - logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName} " + - s"on file ${variable.getName}") - } else { - logInfo(s"Setting secret on environment variable name=${variable.getName}") - } - environment.addVariables(variable) - } - - val command = CommandInfo.newBuilder() - .setEnvironment(environment) - - val uri = conf.get(EXECUTOR_URI).orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) - - if (uri.isEmpty) { - val executorSparkHome = conf.get(EXECUTOR_HOME) - .orElse(sc.getSparkHome()) - .getOrElse { - throw new SparkException(s"Executor Spark home `$EXECUTOR_HOME` is not set!") - } - val executable = new File(executorSparkHome, "./bin/spark-class").getPath - val runScript = "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" - .format(prefixEnv, executable) - - command.setValue(buildExecutorCommand(runScript, taskId, numCores, offer)) - } else { - // Grab everything to the first '.'. We'll use that and '*' to - // glob the directory "correctly". - val basename = uri.get.split('/').last.split('.').head - val runScript = s"cd $basename*; $prefixEnv " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" - - command.setValue(buildExecutorCommand(runScript, taskId, numCores, offer)) - command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get).setCache(useFetcherCache)) - } - - setupUris(conf.get(URIS_TO_DOWNLOAD), command, useFetcherCache) - - command.build() - } - - private def buildExecutorCommand( - runScript: String, taskId: String, numCores: Int, offer: Offer): String = { - - val sb = new StringBuilder() - .append(runScript) - .append(" --driver-url ") - .append(driverURL) - .append(" --executor-id ") - .append(taskId) - .append(" --cores ") - .append(numCores) - .append(" --app-id ") - .append(appId) - - if (sc.conf.get(NETWORK_NAME).isEmpty) { - sb.append(" --hostname ") - sb.append(offer.getHostname) - } - - sb.toString() - } - - protected def driverURL: String = { - if (conf.contains(IS_TESTING)) { - "driverURL" - } else { - RpcEndpointAddress( - conf.get(config.DRIVER_HOST_ADDRESS), - conf.get(config.DRIVER_PORT), - CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString - } - } - - override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID): Unit = {} - - override def registered( - driver: org.apache.mesos.SchedulerDriver, - frameworkId: FrameworkID, - masterInfo: MasterInfo): Unit = { - - this.appId = frameworkId.getValue - this.mesosExternalShuffleClient.foreach(_.init(appId)) - this.schedulerDriver = driver - markRegistered() - launcherBackend.setAppId(appId) - launcherBackend.setState(SparkAppHandle.State.RUNNING) - } - - override def sufficientResourcesRegistered(): Boolean = { - totalCoreCount.get >= maxCoresOption.getOrElse(0) * minRegisteredRatio - } - - override def disconnected(d: org.apache.mesos.SchedulerDriver): Unit = { - launcherBackend.setState(SparkAppHandle.State.SUBMITTED) - } - - override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo): Unit = { - launcherBackend.setState(SparkAppHandle.State.RUNNING) - } - - /** - * Method called by Mesos to offer resources on agents. We respond by launching an executor, - * unless we've already launched more than we wanted to. - */ - override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]): Unit = { - stateLock.synchronized { - if (stopCalled) { - logDebug("Ignoring offers during shutdown") - // Driver should simply return a stopped status on race - // condition between this.stop() and completing here - offers.asScala.map(_.getId).foreach(d.declineOffer) - return - } - - if (numExecutors >= executorLimit) { - logDebug("Executor limit reached. numExecutors: " + numExecutors + - " executorLimit: " + executorLimit) - offers.asScala.map(_.getId).foreach(d.declineOffer) - launchingExecutors = false - return - } else { - if (!launchingExecutors) { - launchingExecutors = true - localityWaitStartTimeNs = System.nanoTime() - } - } - - logDebug(s"Received ${offers.size} resource offers.") - - val (matchedOffers, unmatchedOffers) = offers.asScala.partition { offer => - val offerAttributes = toAttributeMap(offer.getAttributesList) - matchesAttributeRequirements(agentOfferConstraints, offerAttributes) - } - - declineUnmatchedOffers(d, unmatchedOffers) - handleMatchedOffers(d, matchedOffers) - } - } - - private def declineUnmatchedOffers( - driver: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = { - offers.foreach { offer => - declineOffer( - driver, - offer, - Some("unmet constraints"), - Some(rejectOfferDurationForUnmetConstraints)) - } - } - - /** - * Launches executors on accepted offers, and declines unused offers. Executors are launched - * round-robin on offers. - * - * @param driver SchedulerDriver - * @param offers Mesos offers that match attribute constraints - */ - private def handleMatchedOffers( - driver: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = { - val tasks = buildMesosTasks(offers) - for (offer <- offers) { - val offerAttributes = toAttributeMap(offer.getAttributesList) - val offerMem = getResource(offer.getResourcesList, "mem") - val offerCpus = getResource(offer.getResourcesList, "cpus") - val offerPorts = getRangeResource(offer.getResourcesList, "ports") - val offerReservationInfo = offer - .getResourcesList - .asScala - .find { r => r.getReservation != null } - val id = offer.getId.getValue - - if (tasks.contains(offer.getId)) { // accept - val offerTasks = tasks(offer.getId) - - logDebug(s"Accepting offer: $id with attributes: $offerAttributes " + - offerReservationInfo.map(resInfo => - s"reservation info: ${resInfo.getReservation.toString}").getOrElse("") + - s"mem: $offerMem cpu: $offerCpus ports: $offerPorts " + - s"resources: ${offer.getResourcesList.asScala.mkString(",")}." + - s" Launching ${offerTasks.size} Mesos tasks.") - - for (task <- offerTasks) { - val taskId = task.getTaskId - val mem = getResource(task.getResourcesList, "mem") - val cpus = getResource(task.getResourcesList, "cpus") - val ports = getRangeResource(task.getResourcesList, "ports").mkString(",") - - logDebug(s"Launching Mesos task: ${taskId.getValue} with mem: $mem cpu: $cpus" + - s" ports: $ports" + s" on agent with agent id: ${task.getSlaveId.getValue} ") - } - - driver.launchTasks( - Collections.singleton(offer.getId), - offerTasks.asJava) - } else if (totalCoresAcquired >= maxCores) { - // Reject an offer for a configurable amount of time to avoid starving other frameworks - declineOffer(driver, - offer, - Some("reached spark.cores.max"), - Some(rejectOfferDurationForReachedMaxCores)) - } else { - declineOffer( - driver, - offer, - Some("Offer was declined due to unmet task launch constraints.")) - } - } - } - - private def getContainerInfo(conf: SparkConf): ContainerInfo.Builder = { - val containerInfo = MesosSchedulerBackendUtil.buildContainerInfo(conf) - - MesosSchedulerBackendUtil.getSecretVolume(conf, executorSecretConfig).foreach { volume => - if (volume.getSource.getSecret.getReference.isInitialized) { - logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName} " + - s"on file ${volume.getContainerPath}") - } else { - logInfo(s"Setting secret on file name=${volume.getContainerPath}") - } - containerInfo.addVolumes(volume) - } - - containerInfo - } - - /** - * Returns a map from OfferIDs to the tasks to launch on those offers. In order to maximize - * per-task memory and IO, tasks are round-robin assigned to offers. - * - * @param offers Mesos offers that match attribute constraints - * @return A map from OfferID to a list of Mesos tasks to launch on that offer - */ - private def buildMesosTasks(offers: mutable.Buffer[Offer]): Map[OfferID, List[MesosTaskInfo]] = { - // offerID -> tasks - val tasks = new mutable.HashMap[OfferID, List[MesosTaskInfo]].withDefaultValue(Nil) - - // offerID -> resources - val remainingResources = mutable.Map[String, JList[Resource]]() - remainingResources ++= offers.map(offer => (offer.getId.getValue, offer.getResourcesList)) - - var launchTasks = true - - // TODO(mgummelt): combine offers for a single agent - // - // round-robin create executors on the available offers - while (launchTasks) { - launchTasks = false - - for (offer <- offers) { - val agentId = offer.getSlaveId.getValue - val offerId = offer.getId.getValue - val resources = remainingResources(offerId) - - if (canLaunchTask(agentId, offer.getHostname, resources)) { - // Create a task - launchTasks = true - val taskId = newMesosTaskId() - val offerCPUs = getResource(resources, "cpus").toInt - val taskGPUs = Math.min( - Math.max(0, maxGpus - totalGpusAcquired), getResource(resources, "gpus").toInt) - - val taskCPUs = executorCores(offerCPUs) - val taskMemory = executorMemory(sc) - - agents.getOrElseUpdate(agentId, new Agent(offer.getHostname)).taskIDs.add(taskId) - - val (resourcesLeft, resourcesToUse) = - partitionTaskResources(resources, taskCPUs, taskMemory, taskGPUs) - - val taskBuilder = MesosTaskInfo.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(taskId).build()) - .setSlaveId(offer.getSlaveId) - .setCommand(createCommand(offer, taskCPUs + extraCoresPerExecutor, taskId)) - .setName(s"${sc.appName} $taskId") - .setLabels(MesosProtoUtils.mesosLabels(taskLabels)) - .addAllResources(resourcesToUse.asJava) - .setContainer(getContainerInfo(sc.conf)) - - tasks(offer.getId) ::= taskBuilder.build() - remainingResources(offerId) = resourcesLeft.asJava - totalCoresAcquired += taskCPUs - coresByTaskId(taskId) = taskCPUs - if (taskGPUs > 0) { - totalGpusAcquired += taskGPUs - gpusByTaskId(taskId) = taskGPUs - } - } else { - logDebug(s"Cannot launch a task for offer with id: $offerId on agent " + - s"with id: $agentId. Requirements were not met for this offer.") - } - } - } - tasks.toMap - } - - /** Extracts task needed resources from a list of available resources. */ - private def partitionTaskResources( - resources: JList[Resource], - taskCPUs: Int, - taskMemory: Int, - taskGPUs: Int) - : (List[Resource], List[Resource]) = { - - // partition cpus & mem - val (afterCPUResources, cpuResourcesToUse) = partitionResources(resources, "cpus", taskCPUs) - val (afterMemResources, memResourcesToUse) = - partitionResources(afterCPUResources.asJava, "mem", taskMemory) - val (afterGPUResources, gpuResourcesToUse) = - partitionResources(afterMemResources.asJava, "gpus", taskGPUs) - - // If user specifies port numbers in SparkConfig then consecutive tasks will not be launched - // on the same host. This essentially means one executor per host. - // TODO: handle network isolator case - val (nonPortResources, portResourcesToUse) = - partitionPortResources(nonZeroPortValuesFromConfig(sc.conf), afterGPUResources) - - (nonPortResources, - cpuResourcesToUse ++ memResourcesToUse ++ portResourcesToUse ++ gpuResourcesToUse) - } - - private def canLaunchTask(agentId: String, offerHostname: String, - resources: JList[Resource]): Boolean = { - val offerMem = getResource(resources, "mem") - val offerCPUs = getResource(resources, "cpus").toInt - val cpus = executorCores(offerCPUs) - val mem = executorMemory(sc) - val ports = getRangeResource(resources, "ports") - val meetsPortRequirements = checkPorts(sc.conf, ports) - - cpus > 0 && - cpus <= offerCPUs && - cpus + totalCoresAcquired <= maxCores && - mem <= offerMem && - numExecutors < executorLimit && - agents.get(agentId).map(_.taskFailures).getOrElse(0) < MAX_AGENT_FAILURES && - meetsPortRequirements && - satisfiesLocality(offerHostname) - } - - private def executorCores(offerCPUs: Int): Int = { - executorCoresOption.getOrElse( - math.min(offerCPUs, maxCores - totalCoresAcquired) - ) - } - - private def satisfiesLocality(offerHostname: String): Boolean = { - val hostToLocalTaskCount = - rpHostToLocalTaskCount.getOrElse(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, Map.empty) - if (!Utils.isDynamicAllocationEnabled(conf) || hostToLocalTaskCount.isEmpty) { - return true - } - - // Check the locality information - val currentHosts = agents.values.filter(_.taskIDs.nonEmpty).map(_.hostname).toSet - val allDesiredHosts = hostToLocalTaskCount.map { case (k, v) => k }.toSet - - // Try to match locality for hosts which do not have executors yet, to potentially - // increase coverage. - val remainingHosts = allDesiredHosts -- currentHosts - if (!remainingHosts.contains(offerHostname) && - (System.nanoTime() - localityWaitStartTimeNs <= localityWaitNs)) { - logDebug("Skipping host and waiting for locality. host: " + offerHostname) - return false - } - return true - } - - override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus): Unit = { - val taskId = status.getTaskId.getValue - val agentId = status.getSlaveId.getValue - val state = mesosToTaskState(status.getState) - - logInfo(s"Mesos task $taskId is now ${status.getState}") - - stateLock.synchronized { - val agent = agents(agentId) - - // If the shuffle service is enabled, have the driver register with each one of the - // shuffle services. This allows the shuffle services to clean up state associated with - // this application when the driver exits. There is currently not a great way to detect - // this through Mesos, since the shuffle services are set up independently. - if (state.equals(TaskState.RUNNING) && - shuffleServiceEnabled && - !agent.shuffleRegistered) { - assume(mesosExternalShuffleClient.isDefined, - "External shuffle client was not instantiated even though shuffle service is enabled.") - // TODO: Remove this and allow the MesosExternalShuffleService to detect - // framework termination when new Mesos Framework HTTP API is available. - val externalShufflePort = conf.get(config.SHUFFLE_SERVICE_PORT) - - logDebug(s"Connecting to shuffle service on agent $agentId, " + - s"host ${agent.hostname}, port $externalShufflePort for app ${conf.getAppId}") - - mesosExternalShuffleClient.get - .registerDriverWithShuffleService( - agent.hostname, - externalShufflePort, - sc.conf.get( - config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT - ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")), - sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)) - agent.shuffleRegistered = true - } - - if (TaskState.isFinished(state)) { - // Remove the cores we have remembered for this task, if it's in the hashmap - for (cores <- coresByTaskId.get(taskId)) { - totalCoresAcquired -= cores - coresByTaskId -= taskId - } - // Also remove the gpus we have remembered for this task, if it's in the hashmap - for (gpus <- gpusByTaskId.get(taskId)) { - totalGpusAcquired -= gpus - gpusByTaskId -= taskId - } - // If it was a failure, mark the agent as failed for excluding purposes - if (TaskState.isFailed(state)) { - agent.taskFailures += 1 - - if (agent.taskFailures >= MAX_AGENT_FAILURES) { - logInfo(s"Excluding Mesos agent $agentId due to too many failures; " + - "is Spark installed on it?") - } - } - executorTerminated(agentId, taskId, s"Executor finished with state $state") - // In case we'd rejected everything before but have now lost a node - d.reviveOffers() - } - } - } - - override def error(d: org.apache.mesos.SchedulerDriver, message: String): Unit = { - logError(s"Mesos error: $message") - scheduler.error(message) - } - - override def stop(): Unit = { - stopSchedulerBackend() - launcherBackend.setState(SparkAppHandle.State.FINISHED) - launcherBackend.close() - } - - private def stopSchedulerBackend(): Unit = { - // Make sure we're not launching tasks during shutdown - stateLock.synchronized { - if (stopCalled) { - logWarning("Stop called multiple times, ignoring") - return - } - stopCalled = true - super.stop() - } - - // Wait for executors to report done, or else mesosDriver.stop() will forcefully kill them. - // See SPARK-12330 - val startTime = System.nanoTime() - - // agentIdsWithExecutors has no memory barrier, so this is eventually consistent - while (numExecutors() > 0 && - System.nanoTime() - startTime < shutdownTimeoutMS * 1000L * 1000L) { - Thread.sleep(100) - } - - if (numExecutors() > 0) { - logWarning(s"Timed out waiting for ${numExecutors()} remaining executors " - + s"to terminate within $shutdownTimeoutMS ms. This may leave temporary files " - + "on the mesos nodes.") - } - - // Close the mesos external shuffle client if used - mesosExternalShuffleClient.foreach(_.close()) - - if (schedulerDriver != null) { - schedulerDriver.stop() - } - } - - override def frameworkMessage( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: AgentID, b: Array[Byte]): Unit = {} - - /** - * Called when a agent is lost or a Mesos task finished. Updates local view on - * what tasks are running. It also notifies the driver that an executor was removed. - */ - private def executorTerminated( - agentId: String, - taskId: String, - reason: String): Unit = { - stateLock.synchronized { - // Do not call removeExecutor() after this scheduler backend was stopped because - // removeExecutor() internally will send a message to the driver endpoint but - // the driver endpoint is not available now, otherwise an exception will be thrown. - if (!stopCalled) { - removeExecutor(taskId, ExecutorProcessLost(reason)) - } - agents(agentId).taskIDs.remove(taskId) - } - } - - override def agentLost(d: org.apache.mesos.SchedulerDriver, agentId: AgentID): Unit = { - logInfo(s"Mesos agent lost: ${agentId.getValue}") - } - - override def executorLost( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: AgentID, status: Int): Unit = { - logInfo("Mesos executor lost: %s".format(e.getValue)) - } - - override def applicationId(): String = - Option(appId).getOrElse { - logWarning("Application ID is not initialized yet.") - super.applicationId - } - - override def doRequestTotalExecutors( - resourceProfileToTotalExecs: Map[ResourceProfile, Int] - ): Future[Boolean] = Future.successful { - // We don't truly know if we can fulfill the full amount of executors - // since at coarse grain it depends on the amount of agents available. - val numExecs = resourceProfileToTotalExecs.getOrElse(defaultProfile, 0) - logInfo("Capping the total amount of executors to " + numExecs) - executorLimitOption = Some(numExecs) - // Update the locality wait start time to continue trying for locality. - localityWaitStartTimeNs = System.nanoTime() - true - } - - override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future.successful { - if (schedulerDriver == null) { - logWarning("Asked to kill executors before the Mesos driver was started.") - false - } else { - for (executorId <- executorIds) { - val taskId = TaskID.newBuilder().setValue(executorId).build() - schedulerDriver.killTask(taskId) - } - // no need to adjust `executorLimitOption` since the AllocationManager already communicated - // the desired limit through a call to `doRequestTotalExecutors`. - // See [[o.a.s.scheduler.cluster.CoarseGrainedSchedulerBackend.killExecutors]] - true - } - } - - override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { - Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration, driverEndpoint)) - } - - private def numExecutors(): Int = { - agents.values.map(_.taskIDs.size).sum - } -} - -private class Agent(val hostname: String) { - val taskIDs = new mutable.HashSet[String]() - var taskFailures = 0 - var shuffleRegistered = false -} - -object IdHelper { - // Use atomic values since Spark contexts can be initialized in parallel - private[mesos] val nextSCNumber = new AtomicLong(0) - private[mesos] val startedBefore = new AtomicBoolean(false) -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala deleted file mode 100644 index 21df567c8062f..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala +++ /dev/null @@ -1,468 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.io.File -import java.util.{ArrayList => JArrayList, Collections, List => JList} - -import scala.collection.mutable.{HashMap, HashSet} -import scala.jdk.CollectionConverters._ - -import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, SlaveID => AgentID, - TaskInfo => MesosTaskInfo, _} -import org.apache.mesos.SchedulerDriver -import org.apache.mesos.protobuf.ByteString - -import org.apache.spark.{SparkContext, SparkException, TaskState} -import org.apache.spark.deploy.mesos.{config => mesosConfig} -import org.apache.spark.executor.MesosExecutorBackend -import org.apache.spark.internal.config -import org.apache.spark.resource.ResourceProfile -import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.status.api.v1.ThreadStackTrace -import org.apache.spark.util.Utils - -/** - * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a - * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks - * from multiple apps can run on different cores) and in time (a core can switch ownership). - */ -private[spark] class MesosFineGrainedSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext, - master: String) - extends SchedulerBackend - with MesosScheduler - with MesosSchedulerUtils { - - // Stores the agent ids that has launched a Mesos executor. - val agentIdToExecutorInfo = new HashMap[String, MesosExecutorInfo] - val taskIdToAgentId = new HashMap[Long, String] - - // An ExecutorInfo for our tasks - var execArgs: Array[Byte] = null - - var classLoader: ClassLoader = null - - // The listener bus to publish executor added/removed events. - val listenerBus = sc.listenerBus - - private[mesos] val mesosExecutorCores = sc.conf.get(mesosConfig.EXECUTOR_CORES) - - // Offer constraints - private[this] val agentOfferConstraints = - parseConstraintString(sc.conf.get(mesosConfig.CONSTRAINTS)) - - // reject offers with mismatched constraints in seconds - private val rejectOfferDurationForUnmetConstraints = - getRejectOfferDurationForUnmetConstraints(sc.conf) - - private var schedulerDriver: SchedulerDriver = _ - - @volatile var appId: String = _ - - override def start(): Unit = { - classLoader = Thread.currentThread.getContextClassLoader - val driver = createSchedulerDriver( - master, - MesosFineGrainedSchedulerBackend.this, - sc.sparkUser, - sc.appName, - sc.conf, - sc.conf.get(mesosConfig.DRIVER_WEBUI_URL).orElse(sc.ui.map(_.webUrl)), - Option.empty, - Option.empty, - sc.conf.get(mesosConfig.DRIVER_FRAMEWORK_ID) - ) - - unsetFrameworkID(sc) - startScheduler(driver) - } - - /** - * Creates a MesosExecutorInfo that is used to launch a Mesos executor. - * - * @param availableResources Available resources that is offered by Mesos - * @param execId The executor id to assign to this new executor. - * @return A tuple of the new mesos executor info and the remaining available resources. - */ - def createExecutorInfo( - availableResources: JList[Resource], - execId: String): (MesosExecutorInfo, JList[Resource]) = { - val executorSparkHome = sc.conf.get(mesosConfig.EXECUTOR_HOME) - .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility - .getOrElse { - throw new SparkException(s"Executor Spark home `${mesosConfig.EXECUTOR_HOME}` is not set!") - } - val environment = Environment.newBuilder() - sc.conf.get(config.EXECUTOR_CLASS_PATH).foreach { cp => - environment.addVariables( - Environment.Variable.newBuilder().setName("SPARK_EXECUTOR_CLASSPATH").setValue(cp).build()) - } - val extraJavaOpts = sc.conf.get(config.EXECUTOR_JAVA_OPTIONS).map { - Utils.substituteAppNExecIds(_, appId, execId) - }.getOrElse("") - - val prefixEnv = sc.conf.get(config.EXECUTOR_LIBRARY_PATH).map { p => - Utils.libraryPathEnvPrefix(Seq(p)) - }.getOrElse("") - - environment.addVariables( - Environment.Variable.newBuilder() - .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraJavaOpts) - .build()) - sc.executorEnvs.foreach { case (key, value) => - environment.addVariables(Environment.Variable.newBuilder() - .setName(key) - .setValue(value) - .build()) - } - val command = CommandInfo.newBuilder() - .setEnvironment(environment) - val uri = sc.conf.get(mesosConfig.EXECUTOR_URI) - .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) - - val executorBackendName = classOf[MesosExecutorBackend].getName - if (uri.isEmpty) { - val executorPath = new File(executorSparkHome, "/bin/spark-class").getPath - command.setValue(s"$prefixEnv $executorPath $executorBackendName") - } else { - // Grab everything to the first '.'. We'll use that and '*' to - // glob the directory "correctly". - val basename = uri.get.split('/').last.split('.').head - command.setValue(s"cd ${basename}*; $prefixEnv ./bin/spark-class $executorBackendName") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) - } - val builder = MesosExecutorInfo.newBuilder() - val (resourcesAfterCpu, usedCpuResources) = - partitionResources(availableResources, "cpus", mesosExecutorCores) - val (resourcesAfterMem, usedMemResources) = - partitionResources(resourcesAfterCpu.asJava, "mem", executorMemory(sc)) - - builder.addAllResources(usedCpuResources.asJava) - builder.addAllResources(usedMemResources.asJava) - - setupUris(sc.conf.get(mesosConfig.URIS_TO_DOWNLOAD), command) - - val executorInfo = builder - .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) - .setCommand(command) - .setData(ByteString.copyFrom(createExecArg())) - - executorInfo.setContainer( - MesosSchedulerBackendUtil.buildContainerInfo(sc.conf)) - (executorInfo.build(), resourcesAfterMem.asJava) - } - - /** - * Create and serialize the executor argument to pass to Mesos. Our executor arg is an array - * containing all the spark.* system properties in the form of (String, String) pairs. - */ - private def createExecArg(): Array[Byte] = { - if (execArgs == null) { - val props = new HashMap[String, String] - for ((key, value) <- sc.conf.getAll) { - props(key) = value - } - // Serialize the map as an array of (String, String) pairs - execArgs = Utils.serialize(props.toArray) - } - execArgs - } - - override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID): Unit = {} - - override def registered( - driver: org.apache.mesos.SchedulerDriver, - frameworkId: FrameworkID, - masterInfo: MasterInfo): Unit = { - inClassLoader() { - appId = frameworkId.getValue - logInfo("Registered as framework ID " + appId) - this.schedulerDriver = driver - markRegistered() - } - } - - private def inClassLoader()(fun: => Unit) = { - val oldClassLoader = Thread.currentThread.getContextClassLoader - Thread.currentThread.setContextClassLoader(classLoader) - try { - fun - } finally { - Thread.currentThread.setContextClassLoader(oldClassLoader) - } - } - - override def disconnected(d: org.apache.mesos.SchedulerDriver): Unit = {} - - override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo): Unit = {} - - private def getTasksSummary(tasks: JArrayList[MesosTaskInfo]): String = { - val builder = new StringBuilder - tasks.asScala.foreach { t => - builder.append("Task id: ").append(t.getTaskId.getValue).append("\n") - .append("Agent id: ").append(t.getSlaveId.getValue).append("\n") - .append("Task resources: ").append(t.getResourcesList).append("\n") - .append("Executor resources: ").append(t.getExecutor.getResourcesList) - .append("---------------------------------------------\n") - } - builder.toString() - } - - /** - * Method called by Mesos to offer resources on agents. We respond by asking our active task sets - * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that - * tasks are balanced across the cluster. - */ - override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]): Unit = { - inClassLoader() { - // Fail first on offers with unmet constraints - val (offersMatchingConstraints, offersNotMatchingConstraints) = - offers.asScala.partition { o => - val offerAttributes = toAttributeMap(o.getAttributesList) - val meetsConstraints = - matchesAttributeRequirements(agentOfferConstraints, offerAttributes) - - // add some debug messaging - if (!meetsConstraints) { - val id = o.getId.getValue - logDebug(s"Declining offer: $id with attributes: $offerAttributes") - } - - meetsConstraints - } - - // These offers do not meet constraints. We don't need to see them again. - // Decline the offer for a long period of time. - offersNotMatchingConstraints.foreach { o => - d.declineOffer(o.getId, Filters.newBuilder() - .setRefuseSeconds(rejectOfferDurationForUnmetConstraints).build()) - } - - // Of the matching constraints, see which ones give us enough memory and cores - val (usableOffers, unUsableOffers) = offersMatchingConstraints.partition { o => - val mem = getResource(o.getResourcesList, "mem") - val cpus = getResource(o.getResourcesList, "cpus") - val agentId = o.getSlaveId.getValue - val offerAttributes = toAttributeMap(o.getAttributesList) - - // check offers for - // 1. Memory requirements - // 2. CPU requirements - need at least 1 for executor, 1 for task - val meetsMemoryRequirements = mem >= executorMemory(sc) - val meetsCPURequirements = cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK) - val meetsRequirements = - (meetsMemoryRequirements && meetsCPURequirements) || - (agentIdToExecutorInfo.contains(agentId) && cpus >= scheduler.CPUS_PER_TASK) - val debugstr = if (meetsRequirements) "Accepting" else "Declining" - logDebug(s"$debugstr offer: ${o.getId.getValue} with attributes: " - + s"$offerAttributes mem: $mem cpu: $cpus") - - meetsRequirements - } - - // Decline offers we ruled out immediately - unUsableOffers.foreach(o => d.declineOffer(o.getId)) - - val workerOffers = usableOffers.map { o => - val cpus = if (agentIdToExecutorInfo.contains(o.getSlaveId.getValue)) { - getResource(o.getResourcesList, "cpus").toInt - } else { - // If the Mesos executor has not been started on this agent yet, set aside a few - // cores for the Mesos executor by offering fewer cores to the Spark executor - (getResource(o.getResourcesList, "cpus") - mesosExecutorCores).toInt - } - new WorkerOffer( - o.getSlaveId.getValue, - o.getHostname, - cpus) - }.toIndexedSeq - - val agentIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap - val agentIdToWorkerOffer = workerOffers.map(o => o.executorId -> o).toMap - val agentIdToResources = new HashMap[String, JList[Resource]]() - usableOffers.foreach { o => - agentIdToResources(o.getSlaveId.getValue) = o.getResourcesList - } - - val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] - - val agentsIdsOfAcceptedOffers = HashSet[String]() - - // Call into the TaskSchedulerImpl - val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) - acceptedOffers - .foreach { offer => - offer.foreach { taskDesc => - val agentId = taskDesc.executorId - agentsIdsOfAcceptedOffers += agentId - taskIdToAgentId(taskDesc.taskId) = agentId - val (mesosTask, remainingResources) = createMesosTask( - taskDesc, - agentIdToResources(agentId), - agentId) - mesosTasks.getOrElseUpdate(agentId, new JArrayList[MesosTaskInfo]) - .add(mesosTask) - agentIdToResources(agentId) = remainingResources - } - } - - // Reply to the offers - val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? - - mesosTasks.foreach { case (agentId, tasks) => - agentIdToWorkerOffer.get(agentId).foreach(o => - listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), agentId, - // TODO: Add support for log urls for Mesos - new ExecutorInfo(o.host, o.cores, Map.empty, Map.empty))) - ) - logTrace(s"Launching Mesos tasks on agent '$agentId', tasks:\n${getTasksSummary(tasks)}") - d.launchTasks(Collections.singleton(agentIdToOffer(agentId).getId), tasks, filters) - } - - // Decline offers that weren't used - // NOTE: This logic assumes that we only get a single offer for each host in a given batch - for (o <- usableOffers if !agentsIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { - d.declineOffer(o.getId) - } - } - } - - /** Turn a Spark TaskDescription into a Mesos task and also resources unused by the task */ - def createMesosTask( - task: TaskDescription, - resources: JList[Resource], - agentId: String): (MesosTaskInfo, JList[Resource]) = { - val taskId = TaskID.newBuilder().setValue(task.taskId.toString).build() - val (executorInfo, remainingResources) = if (agentIdToExecutorInfo.contains(agentId)) { - (agentIdToExecutorInfo(agentId), resources) - } else { - createExecutorInfo(resources, agentId) - } - agentIdToExecutorInfo(agentId) = executorInfo - val (finalResources, cpuResources) = - partitionResources(remainingResources, "cpus", scheduler.CPUS_PER_TASK) - val taskInfo = MesosTaskInfo.newBuilder() - .setTaskId(taskId) - .setSlaveId(AgentID.newBuilder().setValue(agentId).build()) - .setExecutor(executorInfo) - .setName(task.name) - .addAllResources(cpuResources.asJava) - .setData(ByteString.copyFrom(TaskDescription.encode(task))) - .build() - (taskInfo, finalResources.asJava) - } - - override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus): Unit = { - inClassLoader() { - val tid = status.getTaskId.getValue.toLong - val state = mesosToTaskState(status.getState) - synchronized { - if (TaskState.isFailed(mesosToTaskState(status.getState)) - && taskIdToAgentId.contains(tid)) { - // We lost the executor on this agent, so remember that it's gone - removeExecutor(taskIdToAgentId(tid), "Lost executor") - } - if (TaskState.isFinished(state)) { - taskIdToAgentId.remove(tid) - } - } - scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) - } - } - - override def error(d: org.apache.mesos.SchedulerDriver, message: String): Unit = { - inClassLoader() { - logError("Mesos error: " + message) - markErr() - scheduler.error(message) - } - } - - override def stop(): Unit = { - if (schedulerDriver != null) { - schedulerDriver.stop() - } - } - - override def reviveOffers(): Unit = { - schedulerDriver.reviveOffers() - } - - override def frameworkMessage( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: AgentID, b: Array[Byte]): Unit = {} - - /** - * Remove executor associated with agentId in a thread safe manner. - */ - private def removeExecutor(agentId: String, reason: String) = { - synchronized { - listenerBus.post(SparkListenerExecutorRemoved(System.currentTimeMillis(), agentId, reason)) - agentIdToExecutorInfo -= agentId - } - } - - private def recordAgentLost(agentId: AgentID, reason: ExecutorLossReason): Unit = { - inClassLoader() { - logInfo("Mesos agent lost: " + agentId.getValue) - removeExecutor(agentId.getValue, reason.toString) - scheduler.executorLost(agentId.getValue, reason) - } - } - - override def agentLost(d: org.apache.mesos.SchedulerDriver, agentId: AgentID): Unit = { - recordAgentLost(agentId, ExecutorProcessLost()) - } - - override def executorLost( - d: org.apache.mesos.SchedulerDriver, - executorId: ExecutorID, - agentId: AgentID, - status: Int): Unit = { - logInfo("Executor lost: %s, marking agent %s as lost".format(executorId.getValue, - agentId.getValue)) - recordAgentLost(agentId, ExecutorExited(status, exitCausedByApp = true)) - } - - override def killTask( - taskId: Long, executorId: String, interruptThread: Boolean, reason: String): Unit = { - schedulerDriver.killTask( - TaskID.newBuilder() - .setValue(taskId.toString).build() - ) - } - - // TODO: query Mesos for number of cores - override def defaultParallelism(): Int = sc.conf.getInt("spark.default.parallelism", 8) - - override def applicationId(): String = - Option(appId).getOrElse { - logWarning("Application ID is not initialized yet.") - super.applicationId - } - - override def maxNumConcurrentTasks(rp: ResourceProfile): Int = { - // TODO SPARK-25074 support this method for MesosFineGrainedSchedulerBackend - 0 - } - - override def getTaskThreadDump(taskId: Long, executorId: String): Option[ThreadStackTrace] = None -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosProtoUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosProtoUtils.scala deleted file mode 100644 index 6fd9ad1fa0887..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosProtoUtils.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import scala.jdk.CollectionConverters._ - -import org.apache.mesos.Protos - -import org.apache.spark.SparkException -import org.apache.spark.internal.Logging - -object MesosProtoUtils extends Logging { - - /** Parses a label string of the format specified in spark.mesos.task.labels. */ - def mesosLabels(labelsStr: String): Protos.Labels.Builder = { - val labels: Seq[Protos.Label] = if (labelsStr == "") { - Seq() - } else { - labelsStr.split("""(? - val parts = labelStr.split("""(? part.replaceAll("""\\,""", ",")) - .map(part => part.replaceAll("""\\:""", ":")) - - Protos.Label.newBuilder() - .setKey(cleanedParts(0)) - .setValue(cleanedParts(1)) - .build() - } - } - - Protos.Labels.newBuilder().addAllLabels(labels.asJava) - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosScheduler.scala deleted file mode 100644 index f55b9efb3e64b..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosScheduler.scala +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import org.apache.mesos.Protos.{SlaveID => AgentID} - -trait MesosScheduler extends org.apache.mesos.Scheduler { - override def slaveLost(d: org.apache.mesos.SchedulerDriver, agentId: AgentID): Unit = { - agentLost(d, agentId) - } - - def agentLost(d: org.apache.mesos.SchedulerDriver, agentId: AgentID): Unit -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala deleted file mode 100644 index e9fbff730f07e..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala +++ /dev/null @@ -1,307 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import org.apache.mesos.Protos.{ContainerInfo, Image, NetworkInfo, Parameter, Secret, - TaskState => MesosTaskState, Volume} -import org.apache.mesos.Protos.ContainerInfo.{DockerInfo, MesosInfo} -import org.apache.mesos.Protos.Environment.Variable -import org.apache.mesos.protobuf.ByteString - -import org.apache.spark.{SparkConf, TaskState} -import org.apache.spark.SparkException -import org.apache.spark.deploy.mesos.config._ -import org.apache.spark.deploy.mesos.config.MesosSecretConfig -import org.apache.spark.internal.Logging - -/** - * A collection of utility functions which can be used by the - * MesosSchedulerBackend, [[MesosFineGrainedSchedulerBackend]] and the MesosExecutorBackend. - */ -private[spark] object MesosSchedulerBackendUtil extends Logging { - /** - * Parse a list of volume specs, each of which - * takes the form [host-dir:]container-dir[:rw|:ro]. - */ - def parseVolumesSpec(volumes: Seq[String]): List[Volume] = { - volumes.map(_.split(":")).flatMap { spec => - val vol: Volume.Builder = Volume - .newBuilder() - .setMode(Volume.Mode.RW) - spec match { - case Array(container_path) => - Some(vol.setContainerPath(container_path)) - case Array(container_path, "rw") => - Some(vol.setContainerPath(container_path)) - case Array(container_path, "ro") => - Some(vol.setContainerPath(container_path) - .setMode(Volume.Mode.RO)) - case Array(host_path, container_path) => - Some(vol.setContainerPath(container_path) - .setHostPath(host_path)) - case Array(host_path, container_path, "rw") => - Some(vol.setContainerPath(container_path) - .setHostPath(host_path)) - case Array(host_path, container_path, "ro") => - Some(vol.setContainerPath(container_path) - .setHostPath(host_path) - .setMode(Volume.Mode.RO)) - case spec => - logWarning(s"Unable to parse volume specs: $volumes. " - + "Expected form: \"[host-dir:]container-dir[:rw|:ro](, ...)\"") - None - } - } - .map { _.build() } - .toList - } - - /** - * Parse a list of port mapping specs, each of which - * takes the form host_port:container_port[:udp|:tcp] - * - * Note: - * the docker form is [ip:]host_port:container_port, but the DockerInfo - * message has no field for 'ip', and instead has a 'protocol' field. - * Docker itself only appears to support TCP, so this alternative form - * anticipates the expansion of the docker form to allow for a protocol - * and leaves open the chance for mesos to begin to accept an 'ip' field - */ - def parsePortMappingsSpec(portmaps: Seq[String]): List[DockerInfo.PortMapping] = { - portmaps.map(_.split(":")).flatMap { spec: Array[String] => - val portmap: DockerInfo.PortMapping.Builder = DockerInfo.PortMapping - .newBuilder() - .setProtocol("tcp") - spec match { - case Array(host_port, container_port) => - Some(portmap.setHostPort(host_port.toInt) - .setContainerPort(container_port.toInt)) - case Array(host_port, container_port, protocol) => - Some(portmap.setHostPort(host_port.toInt) - .setContainerPort(container_port.toInt) - .setProtocol(protocol)) - case spec => - logWarning(s"Unable to parse port mapping specs: $portmaps. " - + "Expected form: \"host_port:container_port[:udp|:tcp](, ...)\"") - None - } - } - .map { _.build() } - .toList - } - - /** - * Parse a list of docker parameters, each of which - * takes the form key=value - */ - private def parseParamsSpec(params: Seq[String]): List[Parameter] = { - // split with limit of 2 to avoid parsing error when '=' - // exists in the parameter value - params.map(_.split("=", 2)).flatMap { spec: Array[String] => - val param: Parameter.Builder = Parameter.newBuilder() - spec match { - case Array(key, value) => - Some(param.setKey(key).setValue(value)) - case spec => - logWarning(s"Unable to parse arbitrary parameters: $params. " - + "Expected form: \"key=value(, ...)\"") - None - } - } - .map { _.build() } - .toList - } - - def buildContainerInfo(conf: SparkConf): ContainerInfo.Builder = { - val containerType = if (conf.contains(EXECUTOR_DOCKER_IMAGE) && - conf.get(CONTAINERIZER) == "docker") { - ContainerInfo.Type.DOCKER - } else { - ContainerInfo.Type.MESOS - } - - val containerInfo = ContainerInfo.newBuilder() - .setType(containerType) - - conf.get(EXECUTOR_DOCKER_IMAGE).foreach { image => - val forcePullImage = conf - .get(EXECUTOR_DOCKER_FORCE_PULL_IMAGE).contains(true) - - val portMaps = conf - .get(EXECUTOR_DOCKER_PORT_MAPS) - .map(parsePortMappingsSpec) - .getOrElse(List.empty) - - val params = conf - .get(EXECUTOR_DOCKER_PARAMETERS) - .map(parseParamsSpec) - .getOrElse(List.empty) - - if (containerType == ContainerInfo.Type.DOCKER) { - containerInfo.setDocker( - dockerInfo(image, forcePullImage, portMaps, params, conf.get(NETWORK_NAME)) - ) - } else { - containerInfo.setMesos(mesosInfo(image, forcePullImage)) - } - - val volumes = conf - .get(EXECUTOR_DOCKER_VOLUMES) - .map(parseVolumesSpec) - - volumes.foreach(_.foreach(containerInfo.addVolumes(_))) - } - - conf.get(NETWORK_NAME).map { name => - val networkLabels = MesosProtoUtils.mesosLabels(conf.get(NETWORK_LABELS).getOrElse("")) - val info = NetworkInfo.newBuilder() - .setName(name) - .setLabels(networkLabels) - .build() - containerInfo.addNetworkInfos(info) - } - - containerInfo - } - - private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = { - def createValueSecret(data: String): Secret = { - Secret.newBuilder() - .setType(Secret.Type.VALUE) - .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes))) - .build() - } - - def createReferenceSecret(name: String): Secret = { - Secret.newBuilder() - .setReference(Secret.Reference.newBuilder().setName(name)) - .setType(Secret.Type.REFERENCE) - .build() - } - - val referenceSecrets: Seq[Secret] = - conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map { s => createReferenceSecret(s) } - - val valueSecrets: Seq[Secret] = { - conf.get(secretConfig.SECRET_VALUES).getOrElse(Nil).map { s => createValueSecret(s) } - } - - if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) { - throw new SparkException("Cannot specify both value-type and reference-type secrets.") - } - - if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets - } - - private def illegalSecretInput(dest: Seq[String], secrets: Seq[Secret]): Boolean = { - if (dest.nonEmpty) { - // make sure there is a one-to-one correspondence between destinations and secrets - if (dest.length != secrets.length) { - return true - } - } - false - } - - def getSecretVolume(conf: SparkConf, secretConfig: MesosSecretConfig): List[Volume] = { - val secrets = getSecrets(conf, secretConfig) - val secretPaths: Seq[String] = - conf.get(secretConfig.SECRET_FILENAMES).getOrElse(Nil) - - if (illegalSecretInput(secretPaths, secrets)) { - throw new SparkException( - s"Need to give equal numbers of secrets and file paths for file-based " + - s"reference secrets got secrets $secrets, and paths $secretPaths") - } - - secrets.zip(secretPaths).map { case (s, p) => - val source = Volume.Source.newBuilder() - .setType(Volume.Source.Type.SECRET) - .setSecret(s) - Volume.newBuilder() - .setContainerPath(p) - .setSource(source) - .setMode(Volume.Mode.RO) - .build - }.toList - } - - def getSecretEnvVar(conf: SparkConf, secretConfig: MesosSecretConfig): - List[Variable] = { - val secrets = getSecrets(conf, secretConfig) - val secretEnvKeys = conf.get(secretConfig.SECRET_ENVKEYS).getOrElse(Nil) - if (illegalSecretInput(secretEnvKeys, secrets)) { - throw new SparkException( - s"Need to give equal numbers of secrets and environment keys " + - s"for environment-based reference secrets got secrets $secrets, " + - s"and keys $secretEnvKeys") - } - - secrets.zip(secretEnvKeys).map { case (s, k) => - Variable.newBuilder() - .setName(k) - .setType(Variable.Type.SECRET) - .setSecret(s) - .build - }.toList - } - - private def dockerInfo( - image: String, - forcePullImage: Boolean, - portMaps: List[ContainerInfo.DockerInfo.PortMapping], - params: List[Parameter], - networkName: Option[String]): DockerInfo = { - val dockerBuilder = ContainerInfo.DockerInfo.newBuilder() - .setImage(image) - .setForcePullImage(forcePullImage) - portMaps.foreach(dockerBuilder.addPortMappings(_)) - params.foreach(dockerBuilder.addParameters(_)) - - networkName.foreach { net => - val network = Parameter.newBuilder() - .setKey("net") - .setValue(net) - .build() - - dockerBuilder.setNetwork(DockerInfo.Network.USER) - dockerBuilder.addParameters(network) - } - - dockerBuilder.build - } - - private def mesosInfo(image: String, forcePullImage: Boolean): MesosInfo = { - val imageProto = Image.newBuilder() - .setType(Image.Type.DOCKER) - .setDocker(Image.Docker.newBuilder().setName(image)) - .setCached(!forcePullImage) - ContainerInfo.MesosInfo.newBuilder() - .setImage(imageProto) - .build - } - - def taskStateToMesos(state: TaskState.TaskState): MesosTaskState = state match { - case TaskState.LAUNCHING => MesosTaskState.TASK_STARTING - case TaskState.RUNNING => MesosTaskState.TASK_RUNNING - case TaskState.FINISHED => MesosTaskState.TASK_FINISHED - case TaskState.FAILED => MesosTaskState.TASK_FAILED - case TaskState.KILLED => MesosTaskState.TASK_KILLED - case TaskState.LOST => MesosTaskState.TASK_LOST - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala deleted file mode 100644 index 3702c3f1cf26f..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ /dev/null @@ -1,621 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.io.File -import java.nio.charset.StandardCharsets -import java.util.{List => JList} -import java.util.concurrent.CountDownLatch - -import scala.collection.mutable.ArrayBuffer -import scala.jdk.CollectionConverters._ -import scala.util.control.NonFatal - -import com.google.common.base.Splitter -import com.google.common.io.Files -import org.apache.mesos.{MesosSchedulerDriver, Protos, Scheduler, SchedulerDriver} -import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} -import org.apache.mesos.Protos.FrameworkInfo.Capability -import org.apache.mesos.Protos.Resource.ReservationInfo -import org.apache.mesos.protobuf.GeneratedMessageV3 - -import org.apache.spark.{SparkConf, SparkContext, SparkException} -import org.apache.spark.TaskState -import org.apache.spark.deploy.mesos.{config => mesosConfig, MesosDriverDescription} -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{Status => _, _} -import org.apache.spark.util.Utils - -/** - * Shared trait for implementing a Mesos Scheduler. This holds common state and helper - * methods and Mesos scheduler will use. - */ -trait MesosSchedulerUtils extends Logging { - // Lock used to wait for scheduler to be registered - private final val registerLatch = new CountDownLatch(1) - - private final val ANY_ROLE = "*" - - /** - * Creates a new MesosSchedulerDriver that communicates to the Mesos master. - * - * @param masterUrl The url to connect to Mesos master - * @param scheduler the scheduler class to receive scheduler callbacks - * @param sparkUser User to impersonate with when running tasks - * @param appName The framework name to display on the Mesos UI - * @param conf Spark configuration - * @param webuiUrl The WebUI url to link from Mesos UI - * @param checkpoint Option to checkpoint tasks for failover - * @param failoverTimeout Duration Mesos master expect scheduler to reconnect on disconnect - * @param frameworkId The id of the new framework - */ - protected def createSchedulerDriver( - masterUrl: String, - scheduler: Scheduler, - sparkUser: String, - appName: String, - conf: SparkConf, - webuiUrl: Option[String] = None, - checkpoint: Option[Boolean] = None, - failoverTimeout: Option[Double] = None, - frameworkId: Option[String] = None): SchedulerDriver = { - val fwInfoBuilder = FrameworkInfo.newBuilder().setUser(sparkUser).setName(appName) - fwInfoBuilder.setHostname(Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse( - conf.get(DRIVER_HOST_ADDRESS))) - webuiUrl.foreach { url => fwInfoBuilder.setWebuiUrl(url) } - checkpoint.foreach { checkpoint => fwInfoBuilder.setCheckpoint(checkpoint) } - failoverTimeout.foreach { timeout => fwInfoBuilder.setFailoverTimeout(timeout) } - frameworkId.foreach { id => - fwInfoBuilder.setId(FrameworkID.newBuilder().setValue(id).build()) - } - - conf.get(mesosConfig.ROLE).foreach { role => - fwInfoBuilder.setRole(role) - } - val maxGpus = conf.get(mesosConfig.MAX_GPUS) - if (maxGpus > 0) { - fwInfoBuilder.addCapabilities(Capability.newBuilder().setType(Capability.Type.GPU_RESOURCES)) - } - val credBuilder = buildCredentials(conf, fwInfoBuilder) - if (credBuilder.hasPrincipal) { - new MesosSchedulerDriver( - scheduler, fwInfoBuilder.build(), masterUrl, credBuilder.build()) - } else { - new MesosSchedulerDriver(scheduler, fwInfoBuilder.build(), masterUrl) - } - } - - def buildCredentials( - conf: SparkConf, - fwInfoBuilder: Protos.FrameworkInfo.Builder): Protos.Credential.Builder = { - val credBuilder = Credential.newBuilder() - conf.get(mesosConfig.CREDENTIAL_PRINCIPAL) - .orElse(Option(conf.getenv("SPARK_MESOS_PRINCIPAL"))) - .orElse( - conf.get(mesosConfig.CREDENTIAL_PRINCIPAL_FILE) - .orElse(Option(conf.getenv("SPARK_MESOS_PRINCIPAL_FILE"))) - .map { principalFile => - Files.toString(new File(principalFile), StandardCharsets.UTF_8) - } - ).foreach { principal => - fwInfoBuilder.setPrincipal(principal) - credBuilder.setPrincipal(principal) - } - conf.get(mesosConfig.CREDENTIAL_SECRET) - .orElse(Option(conf.getenv("SPARK_MESOS_SECRET"))) - .orElse( - conf.get(mesosConfig.CREDENTIAL_SECRET_FILE) - .orElse(Option(conf.getenv("SPARK_MESOS_SECRET_FILE"))) - .map { secretFile => - Files.toString(new File(secretFile), StandardCharsets.UTF_8) - } - ).foreach { secret => - credBuilder.setSecret(secret) - } - if (credBuilder.hasSecret && !fwInfoBuilder.hasPrincipal) { - throw new SparkException( - s"${mesosConfig.CREDENTIAL_PRINCIPAL} must be configured when " + - s"${mesosConfig.CREDENTIAL_SECRET} is set") - } - credBuilder - } - - /** - * Starts the MesosSchedulerDriver and stores the current running driver to this new instance. - * This driver is expected to not be running. - * This method returns only after the scheduler has registered with Mesos. - */ - def startScheduler(newDriver: SchedulerDriver): Unit = { - synchronized { - @volatile - var error: Option[Exception] = None - - // We create a new thread that will block inside `mesosDriver.run` - // until the scheduler exists - new Thread(Utils.getFormattedClassName(this) + "-mesos-driver") { - setDaemon(true) - override def run(): Unit = { - try { - val ret = newDriver.run() - logInfo("driver.run() returned with code " + ret) - if (ret != null && ret.equals(Status.DRIVER_ABORTED)) { - error = Some(new SparkException("Error starting driver, DRIVER_ABORTED")) - markErr() - } - } catch { - case e: Exception => - logError("driver.run() failed", e) - error = Some(e) - markErr() - } - } - }.start() - - registerLatch.await() - - // propagate any error to the calling thread. This ensures that SparkContext creation fails - // without leaving a broken context that won't be able to schedule any tasks - error.foreach(throw _) - } - } - - def getResource(res: JList[Resource], name: String): Double = { - // A resource can have multiple values in the offer since it can either be from - // a specific role or wildcard. - res.asScala.filter(_.getName == name).map(_.getScalar.getValue).sum - } - - /** - * Transforms a range resource to a list of ranges - * - * @param res the mesos resource list - * @param name the name of the resource - * @return the list of ranges returned - */ - protected def getRangeResource(res: JList[Resource], name: String): List[(Long, Long)] = { - // A resource can have multiple values in the offer since it can either be from - // a specific role or wildcard. - res.asScala.filter(_.getName == name).flatMap(_.getRanges.getRangeList.asScala - .map(r => (r.getBegin, r.getEnd)).toList).toList - } - - /** - * Signal that the scheduler has registered with Mesos. - */ - protected def markRegistered(): Unit = { - registerLatch.countDown() - } - - protected def markErr(): Unit = { - registerLatch.countDown() - } - - private def setReservationInfo( - reservationInfo: Option[ReservationInfo], - role: Option[String], - builder: Resource.Builder): Unit = { - if (!role.contains(ANY_ROLE)) { - reservationInfo.foreach { res => builder.setReservation(res) } - } - } - - def createResource( - name: String, - amount: Double, - role: Option[String] = None, - reservationInfo: Option[ReservationInfo] = None): Resource = { - val builder = Resource.newBuilder() - .setName(name) - .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(amount).build()) - role.foreach { r => builder.setRole(r) } - setReservationInfo(reservationInfo, role, builder) - builder.build() - } - - private def getReservation(resource: Resource): Option[ReservationInfo] = { - if (resource.hasReservation) { - Some(resource.getReservation) - } else { - None - } - } - /** - * Partition the existing set of resources into two groups, those remaining to be - * scheduled and those requested to be used for a new task. - * - * @param resources The full list of available resources - * @param resourceName The name of the resource to take from the available resources - * @param amountToUse The amount of resources to take from the available resources - * @return The remaining resources list and the used resources list. - */ - def partitionResources( - resources: JList[Resource], - resourceName: String, - amountToUse: Double): (List[Resource], List[Resource]) = { - var remain = amountToUse - var requestedResources = new ArrayBuffer[Resource] - val remainingResources = resources.asScala.map { - case r => - val reservation = getReservation(r) - if (remain > 0 && - r.getType == Value.Type.SCALAR && - r.getScalar.getValue > 0.0 && - r.getName == resourceName) { - val usage = Math.min(remain, r.getScalar.getValue) - requestedResources += createResource(resourceName, usage, - Option(r.getRole), reservation) - remain -= usage - createResource(resourceName, r.getScalar.getValue - usage, - Option(r.getRole), reservation) - } else { - r - } - } - - // Filter any resource that has depleted. - val filteredResources = - remainingResources.filter(r => r.getType != Value.Type.SCALAR || r.getScalar.getValue > 0.0) - - (filteredResources.toList, requestedResources.toList) - } - - /** Helper method to get the key,value-set pair for a Mesos Attribute protobuf */ - protected def getAttribute(attr: Attribute): (String, Set[String]) = { - (attr.getName, attr.getText.getValue.split(',').toSet) - } - - /** - * Converts the attributes from the resource offer into a Map of name to Attribute Value - * The attribute values are the mesos attribute types and they are - * - * @param offerAttributes the attributes offered - */ - protected def toAttributeMap(offerAttributes: JList[Attribute]) - : Map[String, GeneratedMessageV3] = { - offerAttributes.asScala.map { attr => - val attrValue = attr.getType match { - case Value.Type.SCALAR => attr.getScalar - case Value.Type.RANGES => attr.getRanges - case Value.Type.SET => attr.getSet - case Value.Type.TEXT => attr.getText - } - (attr.getName, attrValue) - }.toMap - } - - - /** - * Match the requirements (if any) to the offer attributes. - * if attribute requirements are not specified - return true - * else if attribute is defined and no values are given, simple attribute presence is performed - * else if attribute name and value is specified, subset match is performed on agent attributes - */ - def matchesAttributeRequirements( - agentOfferConstraints: Map[String, Set[String]], - offerAttributes: Map[String, GeneratedMessageV3]): Boolean = { - agentOfferConstraints.forall { - // offer has the required attribute and subsumes the required values for that attribute - case (name, requiredValues) => - offerAttributes.get(name) match { - case Some(_) if requiredValues.isEmpty => true // empty value matches presence - case Some(scalarValue: Value.Scalar) => - // check if provided values is less than equal to the offered values - requiredValues.map(_.toDouble).exists(_ <= scalarValue.getValue) - case Some(rangeValue: Value.Range) => - val offerRange = rangeValue.getBegin to rangeValue.getEnd - // Check if there is some required value that is between the ranges specified - // Note: We only support the ability to specify discrete values, in the future - // we may expand it to subsume ranges specified with a XX..YY value or something - // similar to that. - requiredValues.map(_.toLong).exists(offerRange.contains(_)) - case Some(offeredValue: Value.Set) => - // check if the specified required values is a subset of offered set - requiredValues.subsetOf(offeredValue.getItemList.asScala.toSet) - case Some(textValue: Value.Text) => - // check if the specified value is equal, if multiple values are specified - // we succeed if any of them match. - requiredValues.contains(textValue.getValue) - case _ => false - } - } - } - - /** - * Parses the attributes constraints provided to spark and build a matching data struct: - * {@literal Map[, Set[values-to-match]} - * The constraints are specified as ';' separated key-value pairs where keys and values - * are separated by ':'. The ':' implies equality (for singular values) and "is one of" for - * multiple values (comma separated). For example: - * {{{ - * parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b") - * // would result in - * - * Map( - * "os" -> Set("centos7"), - * "zone": -> Set("us-east-1a", "us-east-1b") - * ) - * }}} - * - * Mesos documentation: http://mesos.apache.org/documentation/attributes-resources/ - * https://github.com/apache/mesos/blob/master/src/common/values.cpp - * https://github.com/apache/mesos/blob/master/src/common/attributes.cpp - * - * @param constraintsVal contains string consisting of ';' separated key-value pairs (separated - * by ':') - * @return Map of constraints to match resources offers. - */ - def parseConstraintString(constraintsVal: String): Map[String, Set[String]] = { - /* - Based on mesos docs: - attributes : attribute ( ";" attribute )* - attribute : labelString ":" ( labelString | "," )+ - labelString : [a-zA-Z0-9_/.-] - */ - val splitter = Splitter.on(';').trimResults().withKeyValueSeparator(':') - // kv splitter - if (constraintsVal.isEmpty) { - Map() - } else { - try { - splitter.split(constraintsVal).asScala.toMap.mapValues(v => - if (v == null || v.isEmpty) { - Set.empty[String] - } else { - v.split(',').toSet - } - ).toMap - } catch { - case NonFatal(e) => - throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e) - } - } - } - - // This default copied from YARN - private val MEMORY_OVERHEAD_MINIMUM = 384 - - /** - * Return the amount of memory to allocate to each executor, taking into account - * container overheads. - * - * @param sc SparkContext to use to get `spark.mesos.executor.memoryOverhead` value - * @return memory requirement as (0.1 * memoryOverhead) or MEMORY_OVERHEAD_MINIMUM - * (whichever is larger) - */ - def executorMemory(sc: SparkContext): Int = { - val memoryOverheadFactor = sc.conf.get(EXECUTOR_MEMORY_OVERHEAD_FACTOR) - sc.conf.get(mesosConfig.EXECUTOR_MEMORY_OVERHEAD).getOrElse( - math.max(memoryOverheadFactor * sc.executorMemory, MEMORY_OVERHEAD_MINIMUM).toInt) + - sc.executorMemory - } - - /** - * Return the amount of memory to allocate to each driver, taking into account - * container overheads. - * - * @param driverDesc used to get driver memory - * @return memory requirement defined as `DRIVER_MEMORY_OVERHEAD` if set in the config, - * otherwise the larger of `MEMORY_OVERHEAD_MINIMUM (=384MB)` or - * `MEMORY_OVERHEAD_FRACTION (=0.1) * driverMemory` - */ - def driverContainerMemory(driverDesc: MesosDriverDescription): Int = { - val memoryOverheadFactor = driverDesc.conf.get(DRIVER_MEMORY_OVERHEAD_FACTOR) - val defaultMem = math.max(memoryOverheadFactor * driverDesc.mem, MEMORY_OVERHEAD_MINIMUM) - driverDesc.conf.get(mesosConfig.DRIVER_MEMORY_OVERHEAD).getOrElse(defaultMem.toInt) + - driverDesc.mem - } - - def setupUris(uris: Seq[String], - builder: CommandInfo.Builder, - useFetcherCache: Boolean = false): Unit = { - uris.foreach { uri => - builder.addUris(CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetcherCache)) - } - } - - protected def getRejectOfferDuration(conf: SparkConf): Long = { - conf.get(mesosConfig.REJECT_OFFER_DURATION) - } - - protected def getRejectOfferDurationForUnmetConstraints(conf: SparkConf): Long = { - conf.get(mesosConfig.REJECT_OFFER_DURATION_FOR_UNMET_CONSTRAINTS) - .getOrElse(getRejectOfferDuration(conf)) - } - - protected def getRejectOfferDurationForReachedMaxCores(conf: SparkConf): Long = { - conf.get(mesosConfig.REJECT_OFFER_DURATION_FOR_REACHED_MAX_CORES) - .getOrElse(getRejectOfferDuration(conf)) - } - - /** - * Checks executor ports if they are within some range of the offered list of ports ranges, - * - * @param conf the Spark Config - * @param ports the list of ports to check - * @return true if ports are within range false otherwise - */ - protected def checkPorts(conf: SparkConf, ports: List[(Long, Long)]): Boolean = { - - def checkIfInRange(port: Long, ps: List[(Long, Long)]): Boolean = { - ps.exists{case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port } - } - - val portsToCheck = nonZeroPortValuesFromConfig(conf) - val withinRange = portsToCheck.forall(p => checkIfInRange(p, ports)) - // make sure we have enough ports to allocate per offer - val enoughPorts = - ports.map{case (rangeStart, rangeEnd) => rangeEnd - rangeStart + 1}.sum >= portsToCheck.size - enoughPorts && withinRange - } - - /** - * Partitions port resources. - * - * @param requestedPorts non-zero ports to assign - * @param offeredResources the resources offered - * @return resources left, port resources to be used. - */ - def partitionPortResources(requestedPorts: List[Long], offeredResources: List[Resource]) - : (List[Resource], List[Resource]) = { - if (requestedPorts.isEmpty) { - (offeredResources, List[Resource]()) - } else { - // partition port offers - val (resourcesWithoutPorts, portResources) = filterPortResources(offeredResources) - - val portsAndResourceInfo = requestedPorts. - map { x => (x, findPortAndGetAssignedResourceInfo(x, portResources)) } - - val assignedPortResources = createResourcesFromPorts(portsAndResourceInfo) - - // ignore non-assigned port resources, they will be declined implicitly by mesos - // no need for splitting port resources. - (resourcesWithoutPorts, assignedPortResources) - } - } - - val managedPortNames = List(BLOCK_MANAGER_PORT.key) - - /** - * The values of the non-zero ports to be used by the executor process. - * - * @param conf the spark config to use - * @return the ono-zero values of the ports - */ - def nonZeroPortValuesFromConfig(conf: SparkConf): List[Long] = { - managedPortNames.map(conf.getLong(_, 0)).filter( _ != 0) - } - - private case class RoleResourceInfo( - role: String, - resInfo: Option[ReservationInfo]) - - /** Creates a mesos resource for a specific port number. */ - private def createResourcesFromPorts( - portsAndResourcesInfo: List[(Long, RoleResourceInfo)]) - : List[Resource] = { - portsAndResourcesInfo.flatMap { case (port, rInfo) => - createMesosPortResource(List((port, port)), Option(rInfo.role), rInfo.resInfo)} - } - - /** Helper to create mesos resources for specific port ranges. */ - private def createMesosPortResource( - ranges: List[(Long, Long)], - role: Option[String] = None, - reservationInfo: Option[ReservationInfo] = None): List[Resource] = { - // for ranges we are going to use (user defined ports fall in there) create mesos resources - // for each range there is a role associated with it. - ranges.map { case (rangeStart, rangeEnd) => - val rangeValue = Value.Range.newBuilder() - .setBegin(rangeStart) - .setEnd(rangeEnd) - val builder = Resource.newBuilder() - .setName("ports") - .setType(Value.Type.RANGES) - .setRanges(Value.Ranges.newBuilder().addRange(rangeValue)) - role.foreach { r => builder.setRole(r) } - setReservationInfo(reservationInfo, role, builder) - builder.build() - } - } - - /** - * Helper to assign a port to an offered range and get the latter's role - * info to use it later on. - */ - private def findPortAndGetAssignedResourceInfo(port: Long, portResources: List[Resource]) - : RoleResourceInfo = { - - val ranges = portResources. - map { resource => - val reservation = getReservation(resource) - (RoleResourceInfo(resource.getRole, reservation), - resource.getRanges.getRangeList.asScala.map(r => (r.getBegin, r.getEnd)).toList) - } - - val rangePortResourceInfo = ranges - .find { case (resourceInfo, rangeList) => rangeList - .exists{ case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port}} - // this is safe since we have previously checked about the ranges (see checkPorts method) - rangePortResourceInfo.map{ case (resourceInfo, rangeList) => resourceInfo}.get - } - - /** Retrieves the port resources from a list of mesos offered resources */ - private def filterPortResources(resources: List[Resource]): (List[Resource], List[Resource]) = { - resources.partition { r => !(r.getType == Value.Type.RANGES && r.getName == "ports") } - } - - /** - * spark.mesos.driver.frameworkId is set by the cluster dispatcher to correlate driver - * submissions with frameworkIDs. However, this causes issues when a driver process launches - * more than one framework (more than one SparkContext(, because they all try to register with - * the same frameworkID. To enforce that only the first driver registers with the configured - * framework ID, the driver calls this method after the first registration. - */ - def unsetFrameworkID(sc: SparkContext): Unit = { - sc.conf.remove(mesosConfig.DRIVER_FRAMEWORK_ID) - System.clearProperty(mesosConfig.DRIVER_FRAMEWORK_ID.key) - } - - def mesosToTaskState(state: MesosTaskState): TaskState.TaskState = state match { - case MesosTaskState.TASK_STAGING | - MesosTaskState.TASK_STARTING => TaskState.LAUNCHING - case MesosTaskState.TASK_RUNNING | - MesosTaskState.TASK_KILLING => TaskState.RUNNING - case MesosTaskState.TASK_FINISHED => TaskState.FINISHED - case MesosTaskState.TASK_FAILED | - MesosTaskState.TASK_GONE | - MesosTaskState.TASK_GONE_BY_OPERATOR => TaskState.FAILED - case MesosTaskState.TASK_KILLED => TaskState.KILLED - case MesosTaskState.TASK_LOST | - MesosTaskState.TASK_ERROR | - MesosTaskState.TASK_DROPPED | - MesosTaskState.TASK_UNKNOWN | - MesosTaskState.TASK_UNREACHABLE => TaskState.LOST - } - - protected def declineOffer( - driver: org.apache.mesos.SchedulerDriver, - offer: Offer, - reason: Option[String] = None, - refuseSeconds: Option[Long] = None): Unit = { - - val id = offer.getId.getValue - val offerAttributes = toAttributeMap(offer.getAttributesList) - val mem = getResource(offer.getResourcesList, "mem") - val cpus = getResource(offer.getResourcesList, "cpus") - val ports = getRangeResource(offer.getResourcesList, "ports") - - logDebug(s"Declining offer: $id with " + - s"attributes: $offerAttributes " + - s"mem: $mem " + - s"cpu: $cpus " + - s"port: $ports " + - refuseSeconds.map(s => s"for ${s} seconds ").getOrElse("") + - reason.map(r => s" (reason: $r)").getOrElse("")) - - refuseSeconds match { - case Some(seconds) => - val filters = Filters.newBuilder().setRefuseSeconds(seconds).build() - driver.declineOffer(offer.getId, filters) - case _ => - driver.declineOffer(offer.getId) - } - } -} diff --git a/resource-managers/mesos/src/test/resources/log4j2.properties b/resource-managers/mesos/src/test/resources/log4j2.properties deleted file mode 100644 index 88847106120cc..0000000000000 --- a/resource-managers/mesos/src/test/resources/log4j2.properties +++ /dev/null @@ -1,31 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Set everything to be logged to the file target/unit-tests.log -rootLogger.level = debug -rootLogger.appenderRef.file.ref = File - -appender.file.type = File -appender.file.name = File -appender.file.fileName = target/unit-tests.log -appender.file.append = true -appender.file.layout.type = PatternLayout -appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n%ex - -# Ignore messages below warning level from Jetty, because it's a bit verbose -logger.jetty.name = org.sparkproject.jetty -logger.jetty.level = warn diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala deleted file mode 100644 index 057c51db455ef..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.TestPrematureExit - -class MesosClusterDispatcherArgumentsSuite extends SparkFunSuite - with TestPrematureExit { - - test("test if spark config args are passed successfully") { - val args = Array[String]("--master", "mesos://localhost:5050", "--conf", "key1=value1", - "--conf", "spark.mesos.key2=value2", "--verbose") - val conf = new SparkConf() - new MesosClusterDispatcherArguments(args, conf) - - assert(conf.getOption("key1").isEmpty) - assert(conf.get("spark.mesos.key2") == "value2") - } - - test("test non conf settings") { - val masterUrl = "mesos://localhost:5050" - val port = "1212" - val zookeeperUrl = "zk://localhost:2181" - val host = "localhost" - val webUiPort = "2323" - val name = "myFramework" - - val args1 = Array("--master", masterUrl, "--verbose", "--name", name) - val args2 = Array("-p", port, "-h", host, "-z", zookeeperUrl) - val args3 = Array("--webui-port", webUiPort) - - val args = args1 ++ args2 ++ args3 - val conf = new SparkConf() - val mesosDispClusterArgs = new MesosClusterDispatcherArguments(args, conf) - - assert(mesosDispClusterArgs.verbose) - assert(mesosDispClusterArgs.confProperties.isEmpty) - assert(mesosDispClusterArgs.host == host) - assert(Option(mesosDispClusterArgs.masterUrl).isDefined) - assert(mesosDispClusterArgs.masterUrl == masterUrl.stripPrefix("mesos://")) - assert(Option(mesosDispClusterArgs.zookeeperUrl).isDefined) - assert(mesosDispClusterArgs.zookeeperUrl == Some(zookeeperUrl)) - assert(mesosDispClusterArgs.name == name) - assert(mesosDispClusterArgs.webUiPort == webUiPort.toInt) - assert(mesosDispClusterArgs.port == port.toInt) - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala deleted file mode 100644 index e6284537402ce..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherSuite.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.TestPrematureExit - -class MesosClusterDispatcherSuite extends SparkFunSuite - with TestPrematureExit { - - test("prints usage on empty input") { - testPrematureExit(Array[String](), - "Usage: MesosClusterDispatcher", MesosClusterDispatcher) - } - - test("prints usage with only --help") { - testPrematureExit(Array("--help"), - "Usage: MesosClusterDispatcher", MesosClusterDispatcher) - } - - test("prints error with unrecognized options") { - testPrematureExit(Array("--blarg"), "Unrecognized option: '--blarg'", MesosClusterDispatcher) - testPrematureExit(Array("-bleg"), "Unrecognized option: '-bleg'", MesosClusterDispatcher) - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.scala deleted file mode 100644 index 8bed43a54d5d0..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.scala +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.rest.mesos - -import javax.servlet.http.HttpServletResponse - -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.TestPrematureExit -import org.apache.spark.deploy.mesos.MesosDriverDescription -import org.apache.spark.deploy.rest.{CreateSubmissionRequest, CreateSubmissionResponse, SubmitRestProtocolMessage, SubmitRestProtocolResponse} -import org.apache.spark.internal.config -import org.apache.spark.scheduler.cluster.mesos.{MesosClusterPersistenceEngineFactory, MesosClusterScheduler} - -class MesosRestServerSuite extends SparkFunSuite - with TestPrematureExit with MockitoSugar { - - test("test default driver overhead memory") { - testOverheadMemory(new SparkConf(), "2000M", 2384) - } - - test("test driver overhead memory with default overhead factor") { - testOverheadMemory(new SparkConf(), "5000M", 5500) - } - - test("test driver overhead memory with overhead factor") { - val conf = new SparkConf() - conf.set(config.DRIVER_MEMORY_OVERHEAD_FACTOR.key, "0.2") - testOverheadMemory(conf, "5000M", 6000) - } - - test("test configured driver overhead memory") { - val conf = new SparkConf() - conf.set(config.DRIVER_MEMORY_OVERHEAD.key, "1000") - testOverheadMemory(conf, "2000M", 3000) - } - - def testOverheadMemory(conf: SparkConf, driverMemory: String, expectedResult: Int): Unit = { - conf.set("spark.master", "testmaster") - conf.set("spark.app.name", "testapp") - conf.set(config.DRIVER_MEMORY.key, driverMemory) - var actualMem = 0 - class TestMesosClusterScheduler extends MesosClusterScheduler( - mock[MesosClusterPersistenceEngineFactory], conf) { - override def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = { - actualMem = desc.mem - mock[CreateSubmissionResponse] - } - } - - class TestServlet extends MesosSubmitRequestServlet(new TestMesosClusterScheduler, conf) { - override def handleSubmit( - requestMessageJson: String, - requestMessage: SubmitRestProtocolMessage, - responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { - super.handleSubmit(requestMessageJson, requestMessage, responseServlet) - } - - override def findUnknownFields( - requestJson: String, - requestMessage: SubmitRestProtocolMessage): Array[String] = { - Array() - } - } - val servlet = new TestServlet() - val request = new CreateSubmissionRequest() - request.appResource = "testresource" - request.mainClass = "mainClass" - request.appArgs = Array("appArgs") - request.environmentVariables = Map("envVar" -> "envVal") - request.sparkProperties = conf.getAll.toMap - servlet.handleSubmit("null", request, null) - assert(actualMem == expectedResult) - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala deleted file mode 100644 index 7f409ae224fc3..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import org.apache.spark._ -import org.apache.spark.deploy.mesos.{config => mesosConfig} -import org.apache.spark.internal.config._ - -class MesosClusterManagerSuite extends SparkFunSuite with LocalSparkContext { - def testURL(masterURL: String, expectedClass: Class[_], coarse: Boolean): Unit = { - val conf = new SparkConf().set(mesosConfig.COARSE_MODE, coarse) - sc = new SparkContext("local", "test", conf) - val clusterManager = new MesosClusterManager() - - assert(clusterManager.canCreate(masterURL)) - val taskScheduler = clusterManager.createTaskScheduler(sc, masterURL) - val sched = clusterManager.createSchedulerBackend(sc, masterURL, taskScheduler) - assert(sched.getClass === expectedClass) - } - - test("mesos fine-grained") { - testURL("mesos://localhost:1234", classOf[MesosFineGrainedSchedulerBackend], coarse = false) - } - - test("mesos coarse-grained") { - testURL("mesos://localhost:1234", classOf[MesosCoarseGrainedSchedulerBackend], coarse = true) - } - - test("mesos with zookeeper") { - testURL("mesos://zk://localhost:1234,localhost:2345", - classOf[MesosFineGrainedSchedulerBackend], - coarse = false) - } - - test("mesos with i/o encryption throws error") { - val se = intercept[SparkException] { - val conf = new SparkConf().setAppName("test").set(IO_ENCRYPTION_ENABLED, true) - sc = new SparkContext("mesos", "test", conf) - } - assert(se.getCause().isInstanceOf[IllegalArgumentException]) - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala deleted file mode 100644 index 8e791d4faf52f..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ /dev/null @@ -1,836 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.util.{Collection, Collections, Date} -import java.util.concurrent.atomic.AtomicLong - -import scala.jdk.CollectionConverters._ - -import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} -import org.apache.mesos.Protos.Value.{Scalar, Type} -import org.apache.mesos.SchedulerDriver -import org.mockito.ArgumentCaptor -import org.mockito.ArgumentMatchers.{eq => meq} -import org.mockito.Mockito._ -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.Command -import org.apache.spark.deploy.mesos.MesosDriverDescription -import org.apache.spark.deploy.mesos.config - -class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { - - private val command = new Command("mainClass", Seq("arg"), Map(), Seq(), Seq(), Seq()) - private var driver: SchedulerDriver = _ - private var scheduler: MesosClusterScheduler = _ - - private val submissionTime = new AtomicLong(System.currentTimeMillis()) - - // Queued drivers in MesosClusterScheduler are ordered based on MesosDriverDescription - // The default ordering checks for priority, followed by submission time. For two driver - // submissions with same priority and if made in quick succession (such that submission - // time is same due to millisecond granularity), this results in dropping the - // second MesosDriverDescription from the queuedDrivers - as driverOrdering - // returns 0 when comparing the descriptions. Ensure two seperate submissions - // have differnt dates - private def getDate: Date = { - new Date(submissionTime.incrementAndGet()) - } - - private def setScheduler(sparkConfVars: Map[String, String] = null): Unit = { - val conf = new SparkConf() - conf.setMaster("mesos://localhost:5050") - conf.setAppName("spark mesos") - - if (sparkConfVars != null) { - conf.setAll(sparkConfVars) - } - - driver = mock[SchedulerDriver] - scheduler = new MesosClusterScheduler( - new BlackHoleMesosClusterPersistenceEngineFactory, conf) { - override def start(): Unit = { ready = true } - } - scheduler.start() - scheduler.registered(driver, Utils.TEST_FRAMEWORK_ID, Utils.TEST_MASTER_INFO) - } - - private def testDriverDescription(submissionId: String): MesosDriverDescription = { - new MesosDriverDescription( - "d1", - "jar", - 1000, - 1, - true, - command, - Map[String, String](), - submissionId, - getDate) - } - - test("can queue drivers") { - setScheduler() - - val response = scheduler.submitDriver(testDriverDescription("s1")) - assert(response.success) - verify(driver, times(1)).reviveOffers() - - val response2 = scheduler.submitDriver(testDriverDescription("s2")) - assert(response2.success) - - val state = scheduler.getSchedulerState() - val queuedDrivers = state.queuedDrivers.toList - assert(queuedDrivers(0).submissionId == response.submissionId) - assert(queuedDrivers(1).submissionId == response2.submissionId) - } - - test("can kill queued drivers") { - setScheduler() - - val response = scheduler.submitDriver(testDriverDescription("s1")) - assert(response.success) - val killResponse = scheduler.killDriver(response.submissionId) - assert(killResponse.success) - val state = scheduler.getSchedulerState() - assert(state.queuedDrivers.isEmpty) - } - - test("can handle multiple roles") { - setScheduler() - - val driver = mock[SchedulerDriver] - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 1200, 1.5, true, - command, - Map((config.EXECUTOR_HOME.key, "test"), ("spark.app.name", "test"), - (config.DRIVER_MEMORY_OVERHEAD.key, "0")), - "s1", - getDate)) - assert(response.success) - val offer = Offer.newBuilder() - .addResources( - Resource.newBuilder().setRole("*") - .setScalar(Scalar.newBuilder().setValue(1).build()).setName("cpus").setType(Type.SCALAR)) - .addResources( - Resource.newBuilder().setRole("*") - .setScalar(Scalar.newBuilder().setValue(1000).build()) - .setName("mem") - .setType(Type.SCALAR)) - .addResources( - Resource.newBuilder().setRole("role2") - .setScalar(Scalar.newBuilder().setValue(1).build()).setName("cpus").setType(Type.SCALAR)) - .addResources( - Resource.newBuilder().setRole("role2") - .setScalar(Scalar.newBuilder().setValue(500).build()).setName("mem").setType(Type.SCALAR)) - .setId(OfferID.newBuilder().setValue("o1").build()) - .setFrameworkId(FrameworkID.newBuilder().setValue("f1").build()) - .setSlaveId(SlaveID.newBuilder().setValue("s1").build()) - .setHostname("host1") - .build() - - val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) - - when( - driver.launchTasks( - meq(Collections.singleton(offer.getId)), - capture.capture()) - ).thenReturn(Status.valueOf(1)) - - scheduler.resourceOffers(driver, Collections.singletonList(offer)) - - val taskInfos = capture.getValue - assert(taskInfos.size() == 1) - val taskInfo = taskInfos.iterator().next() - val resources = taskInfo.getResourcesList - assert(scheduler.getResource(resources, "cpus") == 1.5) - assert(scheduler.getResource(resources, "mem") == 1200) - val resourcesSeq: Seq[Resource] = resources.asScala.toSeq - val cpus = resourcesSeq.filter(_.getName == "cpus").toList - assert(cpus.size == 2) - assert(cpus.exists(_.getRole() == "role2")) - assert(cpus.exists(_.getRole() == "*")) - val mem = resourcesSeq.filter(_.getName == "mem").toList - assert(mem.size == 2) - assert(mem.exists(_.getRole() == "role2")) - assert(mem.exists(_.getRole() == "*")) - - verify(driver, times(1)).launchTasks( - meq(Collections.singleton(offer.getId)), - capture.capture() - ) - } - - test("escapes commandline args for the shell") { - setScheduler() - - val conf = new SparkConf() - conf.setMaster("mesos://localhost:5050") - conf.setAppName("spark mesos") - val scheduler = new MesosClusterScheduler( - new BlackHoleMesosClusterPersistenceEngineFactory, conf) { - override def start(): Unit = { ready = true } - } - val escape = scheduler.shellEscape _ - def wrapped(str: String): String = "\"" + str + "\"" - - // Wrapped in quotes - assert(escape("'should be left untouched'") === "'should be left untouched'") - assert(escape("\"should be left untouched\"") === "\"should be left untouched\"") - - // Harmless - assert(escape("") === "") - assert(escape("harmless") === "harmless") - assert(escape("har-m.l3ss") === "har-m.l3ss") - - // Special Chars escape - assert(escape("should escape this \" quote") === wrapped("should escape this \\\" quote")) - assert(escape("shouldescape\"quote") === wrapped("shouldescape\\\"quote")) - assert(escape("should escape this $ dollar") === wrapped("should escape this \\$ dollar")) - assert(escape("should escape this ` backtick") === wrapped("should escape this \\` backtick")) - assert(escape("""should escape this \ backslash""") - === wrapped("""should escape this \\ backslash""")) - assert(escape("""\"?""") === wrapped("""\\\"?""")) - - - // Special Chars no escape only wrap - List(" ", "'", "<", ">", "&", "|", "?", "*", ";", "!", "#", "(", ")").foreach(char => { - assert(escape(s"onlywrap${char}this") === wrapped(s"onlywrap${char}this")) - }) - } - - test("SPARK-22256: supports spark.mesos.driver.memoryOverhead with 384mb default") { - setScheduler() - - val mem = 1000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map("spark.mesos.executor.home" -> "test", - "spark.app.name" -> "test"), - "s1", - getDate)) - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem*2, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - val tasks = Utils.verifyTaskLaunched(driver, "o1") - // 1384.0 - val taskMem = tasks.head.getResourcesList - .asScala - .filter(_.getName.equals("mem")) - .map(_.getScalar.getValue) - .head - assert(1384.0 === taskMem) - } - - test("SPARK-22256: supports spark.mesos.driver.memoryOverhead with 10% default") { - setScheduler() - - val mem = 10000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map("spark.mesos.executor.home" -> "test", - "spark.app.name" -> "test"), - "s1", - getDate)) - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem*2, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - val tasks = Utils.verifyTaskLaunched(driver, "o1") - // 11000.0 - val taskMem = tasks.head.getResourcesList - .asScala - .filter(_.getName.equals("mem")) - .map(_.getScalar.getValue) - .head - assert(11000.0 === taskMem) - } - - test("supports spark.mesos.driverEnv.*") { - setScheduler() - - val mem = 1000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map(config.EXECUTOR_HOME.key -> "test", - "spark.app.name" -> "test", - config.DRIVER_ENV_PREFIX + "TEST_ENV" -> "TEST_VAL", - config.DRIVER_MEMORY_OVERHEAD.key -> "0" - ), - "s1", - getDate)) - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - val tasks = Utils.verifyTaskLaunched(driver, "o1") - val env = tasks.head.getCommand.getEnvironment.getVariablesList.asScala.map(v => - (v.getName, v.getValue)).toMap - assert(env.getOrElse("TEST_ENV", null) == "TEST_VAL") - } - - test("supports spark.mesos.network.name and spark.mesos.network.labels") { - setScheduler() - - val mem = 1000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map(config.EXECUTOR_HOME.key -> "test", - "spark.app.name" -> "test", - config.NETWORK_NAME.key -> "test-network-name", - config.NETWORK_LABELS.key -> "key1:val1,key2:val2", - config.DRIVER_MEMORY_OVERHEAD.key -> "0"), - "s1", - getDate)) - - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - - val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") - val networkInfos = launchedTasks.head.getContainer.getNetworkInfosList - assert(networkInfos.size == 1) - assert(networkInfos.get(0).getName == "test-network-name") - assert(networkInfos.get(0).getLabels.getLabels(0).getKey == "key1") - assert(networkInfos.get(0).getLabels.getLabels(0).getValue == "val1") - assert(networkInfos.get(0).getLabels.getLabels(1).getKey == "key2") - assert(networkInfos.get(0).getLabels.getLabels(1).getValue == "val2") - } - - test("supports setting fetcher cache") { - setScheduler() - - val mem = 1000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map(config.EXECUTOR_HOME.key -> "test", - config.ENABLE_FETCHER_CACHE.key -> "true", - "spark.app.name" -> "test", - config.DRIVER_MEMORY_OVERHEAD.key -> "0"), - "s1", - getDate)) - - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - - val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") - val uris = launchedTasks.head.getCommand.getUrisList - assert(uris.asScala.forall(_.getCache)) - } - - test("supports setting fetcher cache on the dispatcher") { - setScheduler(Map(config.ENABLE_FETCHER_CACHE.key -> "true")) - - val mem = 1000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map(config.EXECUTOR_HOME.key -> "test", - "spark.app.name" -> "test", - config.DRIVER_MEMORY_OVERHEAD.key -> "0"), - "s1", - getDate)) - - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - - val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") - val uris = launchedTasks.head.getCommand.getUrisList - assert(uris.asScala.forall(_.getCache)) - } - - test("supports disabling fetcher cache") { - setScheduler() - - val mem = 1000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map(config.EXECUTOR_HOME.key -> "test", - config.ENABLE_FETCHER_CACHE.key -> "false", - "spark.app.name" -> "test", - config.DRIVER_MEMORY_OVERHEAD.key -> "0"), - "s1", - getDate)) - - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - - val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") - val uris = launchedTasks.head.getCommand.getUrisList - assert(uris.asScala.forall(!_.getCache)) - } - - test("accept/decline offers with driver constraints") { - setScheduler() - - val mem = 1000 - val cpu = 1 - val s2Attributes = List(Utils.createTextAttribute("c1", "a")) - val s3Attributes = List( - Utils.createTextAttribute("c1", "a"), - Utils.createTextAttribute("c2", "b")) - val offers = List( - Utils.createOffer("o1", "s1", mem, cpu, None, 0), - Utils.createOffer("o2", "s2", mem, cpu, None, 0, s2Attributes), - Utils.createOffer("o3", "s3", mem, cpu, None, 0, s3Attributes)) - - def submitDriver(driverConstraints: String): Unit = { - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map(config.EXECUTOR_HOME.key -> "test", - "spark.app.name" -> "test", - config.DRIVER_CONSTRAINTS.key -> driverConstraints, - config.DRIVER_MEMORY_OVERHEAD.key -> "0"), - "s1", - getDate)) - assert(response.success) - } - - submitDriver("c1:x") - scheduler.resourceOffers(driver, offers.asJava) - offers.foreach(o => Utils.verifyTaskNotLaunched(driver, o.getId.getValue)) - - submitDriver("c1:y;c2:z") - scheduler.resourceOffers(driver, offers.asJava) - offers.foreach(o => Utils.verifyTaskNotLaunched(driver, o.getId.getValue)) - - submitDriver("") - scheduler.resourceOffers(driver, offers.asJava) - Utils.verifyTaskLaunched(driver, "o1") - - submitDriver("c1:a") - scheduler.resourceOffers(driver, offers.asJava) - Utils.verifyTaskLaunched(driver, "o2") - - submitDriver("c1:a;c2:b") - scheduler.resourceOffers(driver, offers.asJava) - Utils.verifyTaskLaunched(driver, "o3") - } - - test("supports spark.mesos.driver.labels") { - setScheduler() - - val mem = 1000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map(config.EXECUTOR_HOME.key -> "test", - "spark.app.name" -> "test", - config.DRIVER_LABELS.key -> "key:value", - config.DRIVER_MEMORY_OVERHEAD.key -> "0"), - "s1", - getDate)) - - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - - val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") - val labels = launchedTasks.head.getLabels - assert(labels.getLabelsCount == 1) - assert(labels.getLabels(0).getKey == "key") - assert(labels.getLabels(0).getValue == "value") - } - - test("can kill supervised drivers") { - val conf = new SparkConf() - conf.setMaster("mesos://localhost:5050") - conf.setAppName("spark mesos") - setScheduler(conf.getAll.toMap) - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 100, 1, true, command, - Map((config.EXECUTOR_HOME.key, "test"), ("spark.app.name", "test")), "s1", getDate)) - assert(response.success) - val agentId = SlaveID.newBuilder().setValue("s1").build() - val offer = Offer.newBuilder() - .addResources( - Resource.newBuilder().setRole("*") - .setScalar(Scalar.newBuilder().setValue(1).build()).setName("cpus").setType(Type.SCALAR)) - .addResources( - Resource.newBuilder().setRole("*") - .setScalar(Scalar.newBuilder().setValue(1000).build()) - .setName("mem") - .setType(Type.SCALAR)) - .setId(OfferID.newBuilder().setValue("o1").build()) - .setFrameworkId(FrameworkID.newBuilder().setValue("f1").build()) - .setSlaveId(agentId) - .setHostname("host1") - .build() - // Offer the resource to launch the submitted driver - scheduler.resourceOffers(driver, Collections.singletonList(offer)) - var state = scheduler.getSchedulerState() - assert(state.launchedDrivers.size == 1) - // Issue the request to kill the launched driver - val killResponse = scheduler.killDriver(response.submissionId) - assert(killResponse.success) - - val taskStatus = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(response.submissionId).build()) - .setSlaveId(agentId) - .setState(MesosTaskState.TASK_KILLED) - .build() - // Update the status of the killed task - scheduler.statusUpdate(driver, taskStatus) - // Driver should be moved to finishedDrivers for kill - state = scheduler.getSchedulerState() - assert(state.pendingRetryDrivers.isEmpty) - assert(state.launchedDrivers.isEmpty) - assert(state.finishedDrivers.size == 1) - } - - test("SPARK-27347: do not restart outdated supervised drivers") { - // Covers scenario where: - // - agent goes down - // - supervised job is relaunched on another agent - // - first agent re-registers and sends status update: TASK_FAILED - // - job should NOT be relaunched again - val conf = new SparkConf() - conf.setMaster("mesos://localhost:5050") - conf.setAppName("SparkMesosDriverRetries") - setScheduler(conf.getAll.toMap) - - val mem = 1000 - val cpu = 1 - val offers = List( - Utils.createOffer("o1", "s1", mem, cpu, None), - Utils.createOffer("o2", "s2", mem, cpu, None), - Utils.createOffer("o3", "s1", mem, cpu, None)) - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 100, 1, true, command, - Map(("spark.mesos.executor.home", "test"), ("spark.app.name", "test")), "sub1", getDate)) - assert(response.success) - - // Offer a resource to launch the submitted driver - scheduler.resourceOffers(driver, Collections.singletonList(offers.head)) - var state = scheduler.getSchedulerState() - assert(state.launchedDrivers.size == 1) - - // Signal agent lost with status with TASK_LOST - val agent1 = SlaveID.newBuilder().setValue("s1").build() - var taskStatus = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(response.submissionId).build()) - .setSlaveId(agent1) - .setReason(TaskStatus.Reason.REASON_SLAVE_REMOVED) - .setState(MesosTaskState.TASK_LOST) - .build() - - scheduler.statusUpdate(driver, taskStatus) - state = scheduler.getSchedulerState() - assert(state.pendingRetryDrivers.size == 1) - assert(state.pendingRetryDrivers.head.submissionId == taskStatus.getTaskId.getValue) - assert(state.launchedDrivers.isEmpty) - - // Offer new resource to retry driver on a new agent - Thread.sleep(1500) // sleep to cover nextRetry's default wait time of 1s - scheduler.resourceOffers(driver, Collections.singletonList(offers(1))) - - val agent2 = SlaveID.newBuilder().setValue("s2").build() - taskStatus = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(response.submissionId).build()) - .setSlaveId(agent2) - .setState(MesosTaskState.TASK_RUNNING) - .build() - - scheduler.statusUpdate(driver, taskStatus) - state = scheduler.getSchedulerState() - assert(state.pendingRetryDrivers.isEmpty) - assert(state.launchedDrivers.size == 1) - assert(state.launchedDrivers.head.taskId.getValue.endsWith("-retry-1")) - assert(state.launchedDrivers.head.taskId.getValue != taskStatus.getTaskId.getValue) - - // Agent1 comes back online and sends status update: TASK_FAILED - taskStatus = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(response.submissionId).build()) - .setSlaveId(agent1) - .setState(MesosTaskState.TASK_FAILED) - .setMessage("Abnormal executor termination") - .setReason(TaskStatus.Reason.REASON_EXECUTOR_TERMINATED) - .build() - - scheduler.statusUpdate(driver, taskStatus) - scheduler.resourceOffers(driver, Collections.singletonList(offers.last)) - - // Assert driver does not restart 2nd time - state = scheduler.getSchedulerState() - assert(state.pendingRetryDrivers.isEmpty) - assert(state.launchedDrivers.size == 1) - assert(state.launchedDrivers.head.taskId.getValue.endsWith("-retry-1")) - } - - test("Declines offer with refuse seconds = 120.") { - setScheduler() - - val filter = Filters.newBuilder().setRefuseSeconds(120).build() - val offerId = OfferID.newBuilder().setValue("o1").build() - val offer = Utils.createOffer(offerId.getValue, "s1", 1000, 1) - - scheduler.resourceOffers(driver, Collections.singletonList(offer)) - - verify(driver, times(1)).declineOffer(offerId, filter) - } - - test("Creates an env-based reference secrets.") { - val launchedTasks = launchDriverTask( - Utils.configEnvBasedRefSecrets(config.driverSecretConfig)) - Utils.verifyEnvBasedRefSecrets(launchedTasks) - } - - test("Creates an env-based value secrets.") { - val launchedTasks = launchDriverTask( - Utils.configEnvBasedValueSecrets(config.driverSecretConfig)) - Utils.verifyEnvBasedValueSecrets(launchedTasks) - } - - test("Creates file-based reference secrets.") { - val launchedTasks = launchDriverTask( - Utils.configFileBasedRefSecrets(config.driverSecretConfig)) - Utils.verifyFileBasedRefSecrets(launchedTasks) - } - - test("Creates a file-based value secrets.") { - val launchedTasks = launchDriverTask( - Utils.configFileBasedValueSecrets(config.driverSecretConfig)) - Utils.verifyFileBasedValueSecrets(launchedTasks) - } - - test("assembles a valid driver command, escaping all confs and args") { - setScheduler() - - val mem = 1000 - val cpu = 1 - val driverDesc = new MesosDriverDescription( - "d1", - "jar", - mem, - cpu, - true, - new Command( - "Main", - Seq("--a=$2", "--b", "x y z"), - Map(), - Seq(), - Seq(), - Seq()), - Map("spark.app.name" -> "app name", - config.EXECUTOR_URI.key -> "s3a://bucket/spark-version.tgz", - "another.conf" -> "\\value"), - "s1", - getDate) - - val expectedCmd = "cd spark-version*; " + - "bin/spark-submit --name \"app name\" --master mesos://mesos://localhost:5050 " + - "--driver-cores 1.0 --driver-memory 1000M --class Main " + - "--conf \"another.conf=\\\\value\" " + - "--conf \"spark.app.name=app name\" " + - "--conf spark.executor.uri=s3a://bucket/spark-version.tgz " + - "../jar " + - "\"--a=\\$2\" " + - "--b \"x y z\"" - - assert(scheduler.getDriverCommandValue(driverDesc) == expectedCmd) - } - - test("SPARK-23499: Test dispatcher priority queue with non float value") { - val conf = new SparkConf() - conf.set("spark.mesos.dispatcher.queue.ROUTINE", "1.0") - conf.set("spark.mesos.dispatcher.queue.URGENT", "abc") - conf.set("spark.mesos.dispatcher.queue.EXCEPTIONAL", "3.0") - assertThrows[NumberFormatException] { - setScheduler(conf.getAll.toMap) - } - } - - test("SPARK-23499: Get driver priority") { - val conf = new SparkConf() - conf.set("spark.mesos.dispatcher.queue.ROUTINE", "1.0") - conf.set("spark.mesos.dispatcher.queue.URGENT", "2.0") - conf.set("spark.mesos.dispatcher.queue.EXCEPTIONAL", "3.0") - setScheduler(conf.getAll.toMap) - - val mem = 1000 - val cpu = 1 - - // Test queue not declared in scheduler - var desc = new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map("spark.mesos.dispatcher.queue" -> "dummy"), - "s1", - getDate) - - assertThrows[NoSuchElementException] { - scheduler.getDriverPriority(desc) - } - - // Test with no specified queue - desc = new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map[String, String](), - "s2", - getDate) - - assert(scheduler.getDriverPriority(desc) == 0.0f) - - // Test with "default" queue specified - desc = new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map("spark.mesos.dispatcher.queue" -> "default"), - "s3", - getDate) - - assert(scheduler.getDriverPriority(desc) == 0.0f) - - // Test queue declared in scheduler - desc = new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map("spark.mesos.dispatcher.queue" -> "ROUTINE"), - "s4", - getDate) - - assert(scheduler.getDriverPriority(desc) == 1.0f) - - // Test other queue declared in scheduler - desc = new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map("spark.mesos.dispatcher.queue" -> "URGENT"), - "s5", - getDate) - - assert(scheduler.getDriverPriority(desc) == 2.0f) - } - - test("SPARK-23499: Can queue drivers with priority") { - val conf = new SparkConf() - conf.set("spark.mesos.dispatcher.queue.ROUTINE", "1.0") - conf.set("spark.mesos.dispatcher.queue.URGENT", "2.0") - conf.set("spark.mesos.dispatcher.queue.EXCEPTIONAL", "3.0") - setScheduler(conf.getAll.toMap) - - val mem = 1000 - val cpu = 1 - - val response0 = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 100, 1, true, command, - Map("spark.mesos.dispatcher.queue" -> "ROUTINE"), "s0", getDate)) - assert(response0.success) - - val response1 = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 100, 1, true, command, - Map[String, String](), "s1", getDate)) - assert(response1.success) - - val response2 = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 100, 1, true, command, - Map("spark.mesos.dispatcher.queue" -> "EXCEPTIONAL"), "s2", getDate)) - assert(response2.success) - - val response3 = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 100, 1, true, command, - Map("spark.mesos.dispatcher.queue" -> "URGENT"), "s3", getDate)) - assert(response3.success) - - val state = scheduler.getSchedulerState() - val queuedDrivers = state.queuedDrivers.toList - assert(queuedDrivers(0).submissionId == response2.submissionId) - assert(queuedDrivers(1).submissionId == response3.submissionId) - assert(queuedDrivers(2).submissionId == response0.submissionId) - assert(queuedDrivers(3).submissionId == response1.submissionId) - } - - test("SPARK-23499: Can queue drivers with negative priority") { - val conf = new SparkConf() - conf.set("spark.mesos.dispatcher.queue.LOWER", "-1.0") - setScheduler(conf.getAll.toMap) - - val mem = 1000 - val cpu = 1 - - val response0 = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 100, 1, true, command, - Map("spark.mesos.dispatcher.queue" -> "LOWER"), "s0", getDate)) - assert(response0.success) - - val response1 = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 100, 1, true, command, - Map[String, String](), "s1", getDate)) - assert(response1.success) - - val state = scheduler.getSchedulerState() - val queuedDrivers = state.queuedDrivers.toList - assert(queuedDrivers(0).submissionId == response1.submissionId) - assert(queuedDrivers(1).submissionId == response0.submissionId) - } - - private def launchDriverTask(addlSparkConfVars: Map[String, String]): List[TaskInfo] = { - setScheduler() - val mem = 1000 - val cpu = 1 - val driverDesc = new MesosDriverDescription( - "d1", - "jar", - mem, - cpu, - true, - command, - Map(config.EXECUTOR_HOME.key -> "test", - "spark.app.name" -> "test", - config.DRIVER_MEMORY_OVERHEAD.key -> "0") ++ - addlSparkConfVars, - "s1", - getDate) - val response = scheduler.submitDriver(driverDesc) - assert(response.success) - val offer = Utils.createOffer("o1", "s1", mem, cpu) - scheduler.resourceOffers(driver, Collections.singletonList(offer)) - Utils.verifyTaskLaunched(driver, "o1") - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala deleted file mode 100644 index 8d0616ad055fe..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ /dev/null @@ -1,843 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.util.concurrent.TimeUnit - -import scala.concurrent.duration._ -import scala.jdk.CollectionConverters._ - -import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} -import org.apache.mesos.Protos._ -import org.mockito.ArgumentMatchers.{any, anyInt, anyLong, anyString, eq => meq} -import org.mockito.Mockito.{times, verify, when} -import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.ScalaFutures -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.deploy.mesos.{config => mesosConfig} -import org.apache.spark.internal.config._ -import org.apache.spark.network.shuffle.mesos.MesosExternalBlockStoreClient -import org.apache.spark.resource.ResourceProfile -import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} -import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor} -import org.apache.spark.scheduler.cluster.mesos.Utils._ - -class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite - with LocalSparkContext - with MockitoSugar - with BeforeAndAfter - with ScalaFutures { - - private var sparkConf: SparkConf = _ - private var driver: SchedulerDriver = _ - private var taskScheduler: TaskSchedulerImpl = _ - private var backend: MesosCoarseGrainedSchedulerBackend = _ - private var externalShuffleClient: MesosExternalBlockStoreClient = _ - private var driverEndpoint: RpcEndpointRef = _ - @volatile private var stopCalled = false - - // All 'requests' to the scheduler run immediately on the same thread, so - // demand that all futures have their value available immediately. - implicit override val patienceConfig = PatienceConfig(timeout = Duration(0, TimeUnit.SECONDS)) - - test("mesos supports killing and limiting executors") { - setBackend() - sparkConf.set(DRIVER_HOST_ADDRESS, "driverHost") - sparkConf.set(DRIVER_PORT, 1234) - - val minMem = backend.executorMemory(sc) - val minCpu = 4 - val offers = List(Resources(minMem, minCpu)) - - // launches a task on a valid offer - offerResources(offers) - verifyTaskLaunched(driver, "o1") - - val totalExecs = Map(ResourceProfile.getOrCreateDefaultProfile(sparkConf) -> 0) - // kills executors - val defaultResourceProfile = ResourceProfile.getOrCreateDefaultProfile(sparkConf) - assert(backend.doRequestTotalExecutors(Map(defaultResourceProfile -> 0)).futureValue) - assert(backend.doKillExecutors(Seq("0")).futureValue) - val taskID0 = createTaskId("0") - verify(driver, times(1)).killTask(taskID0) - - // doesn't launch a new task when requested executors == 0 - offerResources(offers, 2) - verifyDeclinedOffer(driver, createOfferId("o2")) - - // Launches a new task when requested executors is positive - backend.doRequestTotalExecutors(Map(defaultResourceProfile -> 2)) - offerResources(offers, 2) - verifyTaskLaunched(driver, "o2") - } - - test("mesos supports killing and relaunching tasks with executors") { - setBackend() - - // launches a task on a valid offer - val minMem = backend.executorMemory(sc) + 1024 - val minCpu = 4 - val offer1 = Resources(minMem, minCpu) - val offer2 = Resources(minMem, 1) - offerResources(List(offer1, offer2)) - verifyTaskLaunched(driver, "o1") - - // accounts for a killed task - val status = createTaskStatus("0", "s1", TaskState.TASK_KILLED) - backend.statusUpdate(driver, status) - verify(driver, times(1)).reviveOffers() - - // Launches a new task on a valid offer from the same agent - offerResources(List(offer2)) - verifyTaskLaunched(driver, "o2") - } - - test("mesos supports spark.executor.cores") { - val executorCores = 4 - setBackend(Map(EXECUTOR_CORES.key -> executorCores.toString)) - - val executorMemory = backend.executorMemory(sc) - val offers = List(Resources(executorMemory * 2, executorCores + 1)) - offerResources(offers) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus") - assert(cpus == executorCores) - } - - test("mesos supports unset spark.executor.cores") { - setBackend() - - val executorMemory = backend.executorMemory(sc) - val offerCores = 10 - offerResources(List(Resources(executorMemory * 2, offerCores))) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus") - assert(cpus == offerCores) - } - - test("mesos does not acquire more than spark.cores.max") { - val maxCores = 10 - setBackend(Map(CORES_MAX.key -> maxCores.toString)) - - val executorMemory = backend.executorMemory(sc) - offerResources(List(Resources(executorMemory, maxCores + 1))) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus") - assert(cpus == maxCores) - } - - test("mesos does not acquire gpus if not specified") { - setBackend() - - val executorMemory = backend.executorMemory(sc) - offerResources(List(Resources(executorMemory, 1, 1))) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus") - assert(gpus == 0.0) - } - - - test("mesos does not acquire more than spark.mesos.gpus.max") { - val maxGpus = 5 - setBackend(Map(mesosConfig.MAX_GPUS.key -> maxGpus.toString)) - - val executorMemory = backend.executorMemory(sc) - offerResources(List(Resources(executorMemory, 1, maxGpus + 1))) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus") - assert(gpus == maxGpus) - } - - - test("mesos declines offers that violate attribute constraints") { - setBackend(Map(mesosConfig.CONSTRAINTS.key -> "x:true")) - offerResources(List(Resources(backend.executorMemory(sc), 4))) - verifyDeclinedOffer(driver, createOfferId("o1"), true) - } - - test("mesos declines offers with a filter when reached spark.cores.max") { - val maxCores = 3 - setBackend(Map(CORES_MAX.key -> maxCores.toString)) - - val executorMemory = backend.executorMemory(sc) - offerResources(List( - Resources(executorMemory, maxCores + 1), - Resources(executorMemory, maxCores + 1))) - - verifyTaskLaunched(driver, "o1") - verifyDeclinedOffer(driver, createOfferId("o2"), true) - } - - test("mesos declines offers with a filter when maxCores not a multiple of executor.cores") { - val maxCores = 4 - val executorCores = 3 - setBackend(Map( - CORES_MAX.key -> maxCores.toString, - EXECUTOR_CORES.key -> executorCores.toString - )) - val executorMemory = backend.executorMemory(sc) - offerResources(List( - Resources(executorMemory, maxCores + 1), - Resources(executorMemory, maxCores + 1) - )) - verifyTaskLaunched(driver, "o1") - verifyDeclinedOffer(driver, createOfferId("o2"), true) - } - - test("mesos declines offers with a filter when reached spark.cores.max with executor.cores") { - val maxCores = 4 - val executorCores = 2 - setBackend(Map( - CORES_MAX.key -> maxCores.toString, - EXECUTOR_CORES.key -> executorCores.toString - )) - val executorMemory = backend.executorMemory(sc) - offerResources(List( - Resources(executorMemory, maxCores + 1), - Resources(executorMemory, maxCores + 1), - Resources(executorMemory, maxCores + 1) - )) - verifyTaskLaunched(driver, "o1") - verifyTaskLaunched(driver, "o2") - verifyDeclinedOffer(driver, createOfferId("o3"), true) - } - - test("mesos assigns tasks round-robin on offers") { - val executorCores = 4 - val maxCores = executorCores * 2 - setBackend(Map(EXECUTOR_CORES.key -> executorCores.toString, - CORES_MAX.key -> maxCores.toString)) - - val executorMemory = backend.executorMemory(sc) - offerResources(List( - Resources(executorMemory * 2, executorCores * 2), - Resources(executorMemory * 2, executorCores * 2))) - - verifyTaskLaunched(driver, "o1") - verifyTaskLaunched(driver, "o2") - } - - test("mesos creates multiple executors on a single agent") { - val executorCores = 4 - setBackend(Map(EXECUTOR_CORES.key -> executorCores.toString)) - - // offer with room for two executors - val executorMemory = backend.executorMemory(sc) - offerResources(List(Resources(executorMemory * 2, executorCores * 2))) - - // verify two executors were started on a single offer - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 2) - } - - test("mesos doesn't register twice with the same shuffle service") { - setBackend(Map(SHUFFLE_SERVICE_ENABLED.key -> "true")) - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - verifyTaskLaunched(driver, "o1") - - val offer2 = createOffer("o2", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer2).asJava) - verifyTaskLaunched(driver, "o2") - - val status1 = createTaskStatus("0", "s1", TaskState.TASK_RUNNING) - backend.statusUpdate(driver, status1) - - val status2 = createTaskStatus("1", "s1", TaskState.TASK_RUNNING) - backend.statusUpdate(driver, status2) - verify(externalShuffleClient, times(1)) - .registerDriverWithShuffleService(anyString, anyInt, anyLong, anyLong) - } - - test("Port offer decline when there is no appropriate range") { - setBackend(Map(BLOCK_MANAGER_PORT.key -> "30100")) - val offeredPorts = (31100L, 31200L) - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts)) - backend.resourceOffers(driver, List(offer1).asJava) - verify(driver, times(1)).declineOffer(offer1.getId) - } - - test("Port offer accepted when ephemeral ports are used") { - setBackend() - val offeredPorts = (31100L, 31200L) - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts)) - backend.resourceOffers(driver, List(offer1).asJava) - verifyTaskLaunched(driver, "o1") - } - - test("Port offer accepted with user defined port numbers") { - val port = 30100 - setBackend(Map(BLOCK_MANAGER_PORT.key -> s"$port")) - val offeredPorts = (30000L, 31000L) - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts)) - backend.resourceOffers(driver, List(offer1).asJava) - val taskInfo = verifyTaskLaunched(driver, "o1") - - val taskPortResources = taskInfo.head.getResourcesList.asScala. - find(r => r.getType == Value.Type.RANGES && r.getName == "ports") - - val isPortInOffer = (r: Resource) => { - r.getRanges().getRangeList - .asScala.exists(range => range.getBegin == port && range.getEnd == port) - } - assert(taskPortResources.exists(isPortInOffer)) - } - - test("mesos kills an executor when told") { - setBackend() - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - verifyTaskLaunched(driver, "o1") - - backend.doKillExecutors(List("0")) - verify(driver, times(1)).killTask(createTaskId("0")) - } - - test("weburi is set in created scheduler driver") { - initializeSparkConf() - sc = new SparkContext(sparkConf) - - val taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.sc).thenReturn(sc) - - val driver = mock[SchedulerDriver] - when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - - val securityManager = mock[SecurityManager] - - val backend = new MesosCoarseGrainedSchedulerBackend( - taskScheduler, sc, "master", securityManager) { - override protected def createSchedulerDriver( - masterUrl: String, - scheduler: Scheduler, - sparkUser: String, - appName: String, - conf: SparkConf, - webuiUrl: Option[String] = None, - checkpoint: Option[Boolean] = None, - failoverTimeout: Option[Double] = None, - frameworkId: Option[String] = None): SchedulerDriver = { - markRegistered() - assert(webuiUrl.isDefined) - assert(webuiUrl.get.equals("http://webui")) - driver - } - } - - backend.start() - } - - test("failover timeout is set in created scheduler driver") { - val failoverTimeoutIn = 3600.0 - initializeSparkConf(Map(mesosConfig.DRIVER_FAILOVER_TIMEOUT.key -> failoverTimeoutIn.toString)) - sc = new SparkContext(sparkConf) - - val taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.sc).thenReturn(sc) - - val driver = mock[SchedulerDriver] - when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - - val securityManager = mock[SecurityManager] - - val backend = new MesosCoarseGrainedSchedulerBackend( - taskScheduler, sc, "master", securityManager) { - override protected def createSchedulerDriver( - masterUrl: String, - scheduler: Scheduler, - sparkUser: String, - appName: String, - conf: SparkConf, - webuiUrl: Option[String] = None, - checkpoint: Option[Boolean] = None, - failoverTimeout: Option[Double] = None, - frameworkId: Option[String] = None): SchedulerDriver = { - markRegistered() - assert(failoverTimeout.isDefined) - assert(failoverTimeout.get.equals(failoverTimeoutIn)) - driver - } - } - - backend.start() - } - - test("honors unset spark.mesos.containerizer") { - setBackend(Map(mesosConfig.EXECUTOR_DOCKER_IMAGE.key -> "test")) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.DOCKER) - } - - test("honors spark.mesos.containerizer=\"mesos\"") { - setBackend(Map( - mesosConfig.EXECUTOR_DOCKER_IMAGE.key -> "test", - mesosConfig.CONTAINERIZER.key -> "mesos")) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.MESOS) - } - - test("docker settings are reflected in created tasks") { - setBackend(Map( - mesosConfig.EXECUTOR_DOCKER_IMAGE.key -> "some_image", - mesosConfig.EXECUTOR_DOCKER_FORCE_PULL_IMAGE.key -> "true", - mesosConfig.EXECUTOR_DOCKER_VOLUMES.key -> "/host_vol:/container_vol:ro", - mesosConfig.EXECUTOR_DOCKER_PORT_MAPS.key -> "8080:80:tcp" - )) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val launchedTasks = verifyTaskLaunched(driver, "o1") - assert(launchedTasks.size == 1) - - val containerInfo = launchedTasks.head.getContainer - assert(containerInfo.getType == ContainerInfo.Type.DOCKER) - - val volumes = containerInfo.getVolumesList.asScala - assert(volumes.size == 1) - - val volume = volumes.head - assert(volume.getHostPath == "/host_vol") - assert(volume.getContainerPath == "/container_vol") - assert(volume.getMode == Volume.Mode.RO) - - val dockerInfo = containerInfo.getDocker - - val portMappings = dockerInfo.getPortMappingsList.asScala - assert(portMappings.size == 1) - - val portMapping = portMappings.head - assert(portMapping.getHostPort == 8080) - assert(portMapping.getContainerPort == 80) - assert(portMapping.getProtocol == "tcp") - } - - test("force-pull-image option is disabled by default") { - setBackend(Map( - mesosConfig.EXECUTOR_DOCKER_IMAGE.key -> "some_image" - )) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val launchedTasks = verifyTaskLaunched(driver, "o1") - assert(launchedTasks.size == 1) - - val containerInfo = launchedTasks.head.getContainer - assert(containerInfo.getType == ContainerInfo.Type.DOCKER) - - val dockerInfo = containerInfo.getDocker - - assert(dockerInfo.getImage == "some_image") - assert(!dockerInfo.getForcePullImage) - } - - test("mesos supports spark.executor.uri") { - val url = "spark.spark.spark.com" - setBackend(Map( - mesosConfig.EXECUTOR_URI.key -> url - ), null) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val launchedTasks = verifyTaskLaunched(driver, "o1") - assert(launchedTasks.head.getCommand.getUrisList.asScala(0).getValue == url) - } - - test("mesos supports setting fetcher cache") { - val url = "spark.spark.spark.com" - setBackend(Map( - mesosConfig.ENABLE_FETCHER_CACHE.key -> "true", - mesosConfig.EXECUTOR_URI.key -> url - ), null) - val offers = List(Resources(backend.executorMemory(sc), 1)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - val uris = launchedTasks.head.getCommand.getUrisList - assert(uris.size() == 1) - assert(uris.asScala.head.getCache) - } - - test("mesos supports disabling fetcher cache") { - val url = "spark.spark.spark.com" - setBackend(Map( - mesosConfig.ENABLE_FETCHER_CACHE.key -> "false", - mesosConfig.EXECUTOR_URI.key -> url - ), null) - val offers = List(Resources(backend.executorMemory(sc), 1)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - val uris = launchedTasks.head.getCommand.getUrisList - assert(uris.size() == 1) - assert(!uris.asScala.head.getCache) - } - - test("mesos sets task name to spark.app.name") { - setBackend() - - val offers = List(Resources(backend.executorMemory(sc), 1)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - - // Add " 0" to the taskName to match the executor number that is appended - assert(launchedTasks.head.getName == "test-mesos-dynamic-alloc 0") - } - - test("mesos sets configurable labels on tasks") { - val taskLabelsString = "mesos:test,label:test" - setBackend(Map( - mesosConfig.TASK_LABELS.key -> taskLabelsString - )) - - // Build up the labels - val taskLabels = Protos.Labels.newBuilder() - .addLabels(Protos.Label.newBuilder() - .setKey("mesos").setValue("test").build()) - .addLabels(Protos.Label.newBuilder() - .setKey("label").setValue("test").build()) - .build() - - val offers = List(Resources(backend.executorMemory(sc), 1)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - - val labels = launchedTasks.head.getLabels - - assert(launchedTasks.head.getLabels.equals(taskLabels)) - } - - test("mesos supports spark.mesos.network.name and spark.mesos.network.labels") { - setBackend(Map( - mesosConfig.NETWORK_NAME.key -> "test-network-name", - mesosConfig.NETWORK_LABELS.key -> "key1:val1,key2:val2" - )) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val launchedTasks = verifyTaskLaunched(driver, "o1") - val networkInfos = launchedTasks.head.getContainer.getNetworkInfosList - assert(networkInfos.size == 1) - assert(networkInfos.get(0).getName == "test-network-name") - assert(networkInfos.get(0).getLabels.getLabels(0).getKey == "key1") - assert(networkInfos.get(0).getLabels.getLabels(0).getValue == "val1") - assert(networkInfos.get(0).getLabels.getLabels(1).getKey == "key2") - assert(networkInfos.get(0).getLabels.getLabels(1).getValue == "val2") - } - - test("SPARK-28778 '--hostname' shouldn't be set for executor when virtual network is enabled") { - setBackend() - val (mem, cpu) = (backend.executorMemory(sc), 4) - val offer = createOffer("o1", "s1", mem, cpu) - - assert(backend.createCommand(offer, cpu, "test").getValue.contains("--hostname")) - sc.stop() - - setBackend(Map("spark.executor.uri" -> "hdfs://test/executor.jar")) - assert(backend.createCommand(offer, cpu, "test").getValue.contains("--hostname")) - sc.stop() - - setBackend(Map("spark.mesos.network.name" -> "test")) - assert(!backend.createCommand(offer, cpu, "test").getValue.contains("--hostname")) - sc.stop() - - setBackend(Map( - "spark.mesos.network.name" -> "test", - "spark.executor.uri" -> "hdfs://test/executor.jar" - )) - assert(!backend.createCommand(offer, cpu, "test").getValue.contains("--hostname")) - sc.stop() - } - - test("supports spark.scheduler.minRegisteredResourcesRatio") { - val expectedCores = 1 - setBackend(Map( - CORES_MAX.key -> expectedCores.toString, - SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO.key -> "1.0")) - - val offers = List(Resources(backend.executorMemory(sc), expectedCores)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - assert(!backend.isReady) - - registerMockExecutor(launchedTasks(0).getTaskId.getValue, "s1", expectedCores) - assert(backend.isReady) - } - - test("supports data locality with dynamic allocation") { - setBackend(Map( - DYN_ALLOCATION_ENABLED.key -> "true", - DYN_ALLOCATION_TESTING.key -> "true", - LOCALITY_WAIT.key -> "1s")) - - assert(backend.getExecutorIds().isEmpty) - - val defaultProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID - val defaultProf = ResourceProfile.getOrCreateDefaultProfile(sparkConf) - backend.requestTotalExecutors( - Map(defaultProfileId -> 2), - Map(defaultProfileId -> 2), - Map(defaultProfileId -> Map("hosts10" -> 1, "hosts11" -> 1))) - - // Offer non-local resources, which should be rejected - offerResourcesAndVerify(1, false) - offerResourcesAndVerify(2, false) - - // Offer local resource - offerResourcesAndVerify(10, true) - - // Wait longer than spark.locality.wait - Thread.sleep(2000) - - // Offer non-local resource, which should be accepted - offerResourcesAndVerify(1, true) - - // Update total executors - backend.requestTotalExecutors( - Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 3), - Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 2), - Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> - Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1))) - - // Offer non-local resources, which should be rejected - offerResourcesAndVerify(3, false) - - // Wait longer than spark.locality.wait - Thread.sleep(2000) - - // Update total executors - backend.requestTotalExecutors( - Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 4), - Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 4), - Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> - Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1, "hosts13" -> 1))) - - // Offer non-local resources, which should be rejected - offerResourcesAndVerify(3, false) - - // Offer local resource - offerResourcesAndVerify(13, true) - - // Wait longer than spark.locality.wait - Thread.sleep(2000) - - // Offer non-local resource, which should be accepted - offerResourcesAndVerify(2, true) - } - - test("Creates an env-based reference secrets.") { - val launchedTasks = launchExecutorTasks( - configEnvBasedRefSecrets(mesosConfig.executorSecretConfig)) - verifyEnvBasedRefSecrets(launchedTasks) - } - - test("Creates an env-based value secrets.") { - val launchedTasks = launchExecutorTasks( - configEnvBasedValueSecrets(mesosConfig.executorSecretConfig)) - verifyEnvBasedValueSecrets(launchedTasks) - } - - test("Creates file-based reference secrets.") { - val launchedTasks = launchExecutorTasks( - configFileBasedRefSecrets(mesosConfig.executorSecretConfig)) - verifyFileBasedRefSecrets(launchedTasks) - } - - test("Creates a file-based value secrets.") { - val launchedTasks = launchExecutorTasks( - configFileBasedValueSecrets(mesosConfig.executorSecretConfig)) - verifyFileBasedValueSecrets(launchedTasks) - } - - private def launchExecutorTasks(sparkConfVars: Map[String, String]): List[TaskInfo] = { - setBackend(sparkConfVars) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - verifyTaskLaunched(driver, "o1") - } - - private case class Resources(mem: Int, cpus: Int, gpus: Int = 0) - - private def registerMockExecutor(executorId: String, agentId: String, cores: Integer) = { - val mockEndpointRef = mock[RpcEndpointRef] - val mockAddress = mock[RpcAddress] - val message = RegisterExecutor(executorId, mockEndpointRef, agentId, cores, Map.empty, - Map.empty, Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) - - backend.driverEndpoint.askSync[Boolean](message) - } - - private def verifyDeclinedOffer(driver: SchedulerDriver, - offerId: OfferID, - filter: Boolean = false): Unit = { - if (filter) { - verify(driver, times(1)).declineOffer(meq(offerId), any[Filters]()) - } else { - verify(driver, times(1)).declineOffer(meq(offerId)) - } - } - - private def offerResources(offers: List[Resources], startId: Int = 1): Unit = { - val mesosOffers = offers.zipWithIndex.map {case (offer, i) => - createOffer(s"o${i + startId}", s"s${i + startId}", offer.mem, offer.cpus, None, offer.gpus)} - - backend.resourceOffers(driver, mesosOffers.asJava) - } - - private def offerResourcesAndVerify(id: Int, expectAccept: Boolean): Unit = { - offerResources(List(Resources(backend.executorMemory(sc), 1)), id) - if (expectAccept) { - val numExecutors = backend.getExecutorIds().size - val launchedTasks = verifyTaskLaunched(driver, s"o$id") - assert(s"s$id" == launchedTasks.head.getSlaveId.getValue) - registerMockExecutor(launchedTasks.head.getTaskId.getValue, s"s$id", 1) - assert(backend.getExecutorIds().size == numExecutors + 1) - } else { - verifyTaskNotLaunched(driver, s"o$id") - } - } - - private def createTaskStatus(taskId: String, agentId: String, state: TaskState): TaskStatus = { - TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(taskId).build()) - .setSlaveId(SlaveID.newBuilder().setValue(agentId).build()) - .setState(state) - .build - } - - private def createSchedulerBackend( - taskScheduler: TaskSchedulerImpl, - driver: SchedulerDriver, - shuffleClient: MesosExternalBlockStoreClient) = { - val securityManager = mock[SecurityManager] - - val backend = new MesosCoarseGrainedSchedulerBackend( - taskScheduler, sc, "master", securityManager) { - override protected def createSchedulerDriver( - masterUrl: String, - scheduler: Scheduler, - sparkUser: String, - appName: String, - conf: SparkConf, - webuiUrl: Option[String] = None, - checkpoint: Option[Boolean] = None, - failoverTimeout: Option[Double] = None, - frameworkId: Option[String] = None): SchedulerDriver = driver - - override protected def getShuffleClient(): MesosExternalBlockStoreClient = shuffleClient - - // override to avoid race condition with the driver thread on `mesosDriver` - override def startScheduler(newDriver: SchedulerDriver): Unit = {} - - override def stopExecutors(): Unit = { - stopCalled = true - } - } - backend.start() - backend.registered(driver, Utils.TEST_FRAMEWORK_ID, Utils.TEST_MASTER_INFO) - backend - } - - private def initializeSparkConf( - sparkConfVars: Map[String, String] = null, - home: String = "/path"): Unit = { - sparkConf = (new SparkConf) - .setMaster("local[*]") - .setAppName("test-mesos-dynamic-alloc") - .set(mesosConfig.DRIVER_WEBUI_URL, "http://webui") - - if (home != null) { - sparkConf.setSparkHome(home) - } - - if (sparkConfVars != null) { - sparkConf.setAll(sparkConfVars) - } - } - - private def setBackend(sparkConfVars: Map[String, String] = null, - home: String = "/path"): Unit = { - initializeSparkConf(sparkConfVars, home) - sc = new SparkContext(sparkConf) - - driver = mock[SchedulerDriver] - when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - - taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.excludedNodes).thenReturn(Set[String]()) - when(taskScheduler.sc).thenReturn(sc) - - externalShuffleClient = mock[MesosExternalBlockStoreClient] - - backend = createSchedulerBackend(taskScheduler, driver, externalShuffleClient) - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala deleted file mode 100644 index 1b417f5b30b94..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala +++ /dev/null @@ -1,420 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.nio.ByteBuffer -import java.util.Arrays -import java.util.Collection -import java.util.Collections -import java.util.Properties - -import scala.collection.immutable -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer -import scala.jdk.CollectionConverters._ - -import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} -import org.apache.mesos.Protos._ -import org.apache.mesos.Protos.Value.Scalar -import org.mockito.ArgumentCaptor -import org.mockito.ArgumentMatchers.{any, anyLong, eq => meq} -import org.mockito.Mockito._ -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.{JobArtifactSet, LocalSparkContext, SparkConf, SparkContext, - SparkFunSuite} -import org.apache.spark.deploy.mesos.config._ -import org.apache.spark.executor.MesosExecutorBackend -import org.apache.spark.resource.ResourceInformation -import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, - TaskDescription, TaskSchedulerImpl, WorkerOffer} -import org.apache.spark.scheduler.cluster.ExecutorInfo - -class MesosFineGrainedSchedulerBackendSuite - extends SparkFunSuite with LocalSparkContext with MockitoSugar { - - test("weburi is set in created scheduler driver") { - val conf = new SparkConf - conf.set(DRIVER_WEBUI_URL, "http://webui") - conf.set("spark.app.name", "name1") - - val sc = mock[SparkContext] - when(sc.conf).thenReturn(conf) - when(sc.sparkUser).thenReturn("sparkUser1") - when(sc.appName).thenReturn("appName1") - - val taskScheduler = mock[TaskSchedulerImpl] - val driver = mock[SchedulerDriver] - when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - - val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") { - override protected def createSchedulerDriver( - masterUrl: String, - scheduler: Scheduler, - sparkUser: String, - appName: String, - conf: SparkConf, - webuiUrl: Option[String] = None, - checkpoint: Option[Boolean] = None, - failoverTimeout: Option[Double] = None, - frameworkId: Option[String] = None): SchedulerDriver = { - markRegistered() - assert(webuiUrl.isDefined) - assert(webuiUrl.get.equals("http://webui")) - driver - } - } - - backend.start() - } - - test("Use configured mesosExecutor.cores for ExecutorInfo") { - val mesosExecutorCores = 3.0 - val conf = new SparkConf - conf.set(EXECUTOR_CORES, mesosExecutorCores) - - val listenerBus = mock[LiveListenerBus] - listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", - new ExecutorInfo("host1", 2, Map.empty, Map.empty))) - - val sc = mock[SparkContext] - when(sc.getSparkHome()).thenReturn(Option("/spark-home")) - - when(sc.conf).thenReturn(conf) - when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) - when(sc.executorMemory).thenReturn(100) - when(sc.listenerBus).thenReturn(listenerBus) - val taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - - val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") - - val resources = Arrays.asList( - mesosSchedulerBackend.createResource("cpus", 4), - mesosSchedulerBackend.createResource("mem", 1024)) - // uri is null. - val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") - val executorResources = executorInfo.getResourcesList - val cpus = executorResources.asScala.find(_.getName == "cpus").get.getScalar.getValue - - assert(cpus === mesosExecutorCores) - } - - test("check spark-class location correctly") { - val conf = new SparkConf - conf.set(EXECUTOR_HOME, "/mesos-home") - - val listenerBus = mock[LiveListenerBus] - listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", - new ExecutorInfo("host1", 2, Map.empty, Map.empty))) - - val sc = mock[SparkContext] - when(sc.getSparkHome()).thenReturn(Option("/spark-home")) - - when(sc.conf).thenReturn(conf) - when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) - when(sc.executorMemory).thenReturn(100) - when(sc.listenerBus).thenReturn(listenerBus) - val taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - - val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") - - val resources = Arrays.asList( - mesosSchedulerBackend.createResource("cpus", 4), - mesosSchedulerBackend.createResource("mem", 1024)) - // uri is null. - val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") - assert(executorInfo.getCommand.getValue === - s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") - - // uri exists. - conf.set(EXECUTOR_URI, "hdfs:///test-app-1.0.0.tgz") - val (executorInfo1, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") - assert(executorInfo1.getCommand.getValue === - s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") - } - - test("spark docker properties correctly populate the DockerInfo message") { - val taskScheduler = mock[TaskSchedulerImpl] - - val conf = new SparkConf() - .set(EXECUTOR_DOCKER_IMAGE, "spark/mock") - .set(EXECUTOR_DOCKER_FORCE_PULL_IMAGE, true) - .set(EXECUTOR_DOCKER_VOLUMES, Seq("/a", "/b:/b", "/c:/c:rw", "/d:ro", "/e:/e:ro")) - .set(EXECUTOR_DOCKER_PORT_MAPS, Seq("80:8080", "53:53:tcp")) - - val listenerBus = mock[LiveListenerBus] - listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", - new ExecutorInfo("host1", 2, Map.empty, Map.empty))) - - val sc = mock[SparkContext] - when(sc.executorMemory).thenReturn(100) - when(sc.getSparkHome()).thenReturn(Option("/spark-home")) - when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) - when(sc.conf).thenReturn(conf) - when(sc.listenerBus).thenReturn(listenerBus) - - val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") - - val (execInfo, _) = backend.createExecutorInfo( - Arrays.asList(backend.createResource("cpus", 4)), "mockExecutor") - assert(execInfo.getContainer.getDocker.getImage.equals("spark/mock")) - assert(execInfo.getContainer.getDocker.getForcePullImage) - val portmaps = execInfo.getContainer.getDocker.getPortMappingsList - assert(portmaps.get(0).getHostPort.equals(80)) - assert(portmaps.get(0).getContainerPort.equals(8080)) - assert(portmaps.get(0).getProtocol.equals("tcp")) - assert(portmaps.get(1).getHostPort.equals(53)) - assert(portmaps.get(1).getContainerPort.equals(53)) - assert(portmaps.get(1).getProtocol.equals("tcp")) - val volumes = execInfo.getContainer.getVolumesList - assert(volumes.get(0).getContainerPath.equals("/a")) - assert(volumes.get(0).getMode.equals(Volume.Mode.RW)) - assert(volumes.get(1).getContainerPath.equals("/b")) - assert(volumes.get(1).getHostPath.equals("/b")) - assert(volumes.get(1).getMode.equals(Volume.Mode.RW)) - assert(volumes.get(2).getContainerPath.equals("/c")) - assert(volumes.get(2).getHostPath.equals("/c")) - assert(volumes.get(2).getMode.equals(Volume.Mode.RW)) - assert(volumes.get(3).getContainerPath.equals("/d")) - assert(volumes.get(3).getMode.equals(Volume.Mode.RO)) - assert(volumes.get(4).getContainerPath.equals("/e")) - assert(volumes.get(4).getHostPath.equals("/e")) - assert(volumes.get(4).getMode.equals(Volume.Mode.RO)) - } - - test("mesos resource offers result in launching tasks") { - def createOffer(id: Int, mem: Int, cpu: Int): Offer = { - val builder = Offer.newBuilder() - builder.addResourcesBuilder() - .setName("mem") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(mem)) - builder.addResourcesBuilder() - .setName("cpus") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()) - .setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")) - .setHostname(s"host${id.toString}").build() - } - - val driver = mock[SchedulerDriver] - val taskScheduler = mock[TaskSchedulerImpl] - - val listenerBus = mock[LiveListenerBus] - listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", - new ExecutorInfo("host1", 2, Map.empty, Map.empty))) - - val sc = mock[SparkContext] - when(sc.executorMemory).thenReturn(100) - when(sc.getSparkHome()).thenReturn(Option("/path")) - when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) - when(sc.conf).thenReturn(new SparkConf) - when(sc.listenerBus).thenReturn(listenerBus) - - val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") - - val minMem = backend.executorMemory(sc) - val minCpu = 4 - - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(createOffer(1, minMem, minCpu)) - mesosOffers.add(createOffer(2, minMem - 1, minCpu)) - mesosOffers.add(createOffer(3, minMem, minCpu)) - - val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) - expectedWorkerOffers += new WorkerOffer( - mesosOffers.get(0).getSlaveId.getValue, - mesosOffers.get(0).getHostname, - (minCpu - backend.mesosExecutorCores).toInt - ) - expectedWorkerOffers += new WorkerOffer( - mesosOffers.get(2).getSlaveId.getValue, - mesosOffers.get(2).getHostname, - (minCpu - backend.mesosExecutorCores).toInt - ) - val taskDesc = new TaskDescription( - taskId = 1L, - attemptNumber = 0, - executorId = "s1", - name = "n1", - index = 0, - partitionId = 0, - artifacts = JobArtifactSet.emptyJobArtifactSet, - properties = new Properties(), - cpus = 1, - resources = immutable.Map.empty[String, ResourceInformation], - ByteBuffer.wrap(new Array[Byte](0))) - when(taskScheduler.resourceOffers( - expectedWorkerOffers.toIndexedSeq)).thenReturn(Seq(Seq(taskDesc))) - when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - - val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) - when( - driver.launchTasks( - meq(Collections.singleton(mesosOffers.get(0).getId)), - capture.capture(), - any(classOf[Filters]) - ) - ).thenReturn(Status.valueOf(1)) - when(driver.declineOffer(mesosOffers.get(1).getId)).thenReturn(Status.valueOf(1)) - when(driver.declineOffer(mesosOffers.get(2).getId)).thenReturn(Status.valueOf(1)) - - backend.resourceOffers(driver, mesosOffers) - - verify(driver, times(1)).launchTasks( - meq(Collections.singleton(mesosOffers.get(0).getId)), - capture.capture(), - any(classOf[Filters]) - ) - verify(driver, times(1)).declineOffer(mesosOffers.get(1).getId) - verify(driver, times(1)).declineOffer(mesosOffers.get(2).getId) - assert(capture.getValue.size() === 1) - val taskInfo = capture.getValue.iterator().next() - assert(taskInfo.getName.equals("n1")) - val cpus = taskInfo.getResourcesList.get(0) - assert(cpus.getName.equals("cpus")) - assert(cpus.getScalar.getValue.equals(2.0)) - assert(taskInfo.getSlaveId.getValue.equals("s1")) - - // Unwanted resources offered on an existing node. Make sure they are declined - val mesosOffers2 = new java.util.ArrayList[Offer] - mesosOffers2.add(createOffer(1, minMem, minCpu)) - reset(taskScheduler) - reset(driver) - when(taskScheduler.resourceOffers(any(classOf[IndexedSeq[WorkerOffer]]), any[Boolean])) - .thenReturn(Seq(Seq())) - when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - when(driver.declineOffer(mesosOffers2.get(0).getId)).thenReturn(Status.valueOf(1)) - - backend.resourceOffers(driver, mesosOffers2) - verify(driver, times(1)).declineOffer(mesosOffers2.get(0).getId) - } - - test("can handle multiple roles") { - val driver = mock[SchedulerDriver] - val taskScheduler = mock[TaskSchedulerImpl] - - val listenerBus = mock[LiveListenerBus] - listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", - new ExecutorInfo("host1", 2, Map.empty, Map.empty))) - - val sc = mock[SparkContext] - when(sc.executorMemory).thenReturn(100) - when(sc.getSparkHome()).thenReturn(Option("/path")) - when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) - when(sc.conf).thenReturn(new SparkConf) - when(sc.listenerBus).thenReturn(listenerBus) - - val id = 1 - val builder = Offer.newBuilder() - builder.addResourcesBuilder() - .setName("mem") - .setType(Value.Type.SCALAR) - .setRole("prod") - .setScalar(Scalar.newBuilder().setValue(500)) - builder.addResourcesBuilder() - .setName("cpus") - .setRole("prod") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(1)) - builder.addResourcesBuilder() - .setName("mem") - .setRole("dev") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(600)) - builder.addResourcesBuilder() - .setName("cpus") - .setRole("dev") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(2)) - val offer = builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()) - .setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")) - .setHostname(s"host${id.toString}").build() - - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(offer) - - val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") - - val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](1) - expectedWorkerOffers += new WorkerOffer( - mesosOffers.get(0).getSlaveId.getValue, - mesosOffers.get(0).getHostname, - 2 // Deducting 1 for executor - ) - - val taskDesc = new TaskDescription( - taskId = 1L, - attemptNumber = 0, - executorId = "s1", - name = "n1", - index = 0, - partitionId = 0, - artifacts = JobArtifactSet.emptyJobArtifactSet, - properties = new Properties(), - cpus = 1, - resources = immutable.Map.empty[String, ResourceInformation], - ByteBuffer.wrap(new Array[Byte](0))) - when(taskScheduler.resourceOffers( - expectedWorkerOffers.toIndexedSeq)).thenReturn(Seq(Seq(taskDesc))) - when(taskScheduler.CPUS_PER_TASK).thenReturn(1) - - val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) - when( - driver.launchTasks( - meq(Collections.singleton(mesosOffers.get(0).getId)), - capture.capture(), - any(classOf[Filters]) - ) - ).thenReturn(Status.valueOf(1)) - - backend.resourceOffers(driver, mesosOffers) - - verify(driver, times(1)).launchTasks( - meq(Collections.singleton(mesosOffers.get(0).getId)), - capture.capture(), - any(classOf[Filters]) - ) - - assert(capture.getValue.size() === 1) - val taskInfo = capture.getValue.iterator().next() - assert(taskInfo.getName.equals("n1")) - assert(taskInfo.getResourcesCount === 1) - val cpusDev = taskInfo.getResourcesList.get(0) - assert(cpusDev.getName.equals("cpus")) - assert(cpusDev.getScalar.getValue.equals(1.0)) - assert(cpusDev.getRole.equals("dev")) - val executorResources = taskInfo.getExecutor.getResourcesList.asScala - assert(executorResources.exists { r => - r.getName.equals("mem") && r.getScalar.getValue.equals(484.0) && r.getRole.equals("prod") - }) - assert(executorResources.exists { r => - r.getName.equals("cpus") && r.getScalar.getValue.equals(1.0) && r.getRole.equals("prod") - }) - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosProtoUtilsSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosProtoUtilsSuite.scala deleted file mode 100644 index 36a4c1ab1ad25..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosProtoUtilsSuite.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import org.apache.spark.SparkFunSuite - -class MesosProtoUtilsSuite extends SparkFunSuite { - test("mesosLabels") { - val labels = MesosProtoUtils.mesosLabels("key:value") - assert(labels.getLabelsCount == 1) - val label = labels.getLabels(0) - assert(label.getKey == "key") - assert(label.getValue == "value") - - val labels2 = MesosProtoUtils.mesosLabels("key:value\\:value") - assert(labels2.getLabelsCount == 1) - val label2 = labels2.getLabels(0) - assert(label2.getKey == "key") - assert(label2.getValue == "value:value") - - val labels3 = MesosProtoUtils.mesosLabels("key:value,key2:value2") - assert(labels3.getLabelsCount == 2) - assert(labels3.getLabels(0).getKey == "key") - assert(labels3.getLabels(0).getValue == "value") - assert(labels3.getLabels(1).getKey == "key2") - assert(labels3.getLabels(1).getValue == "value2") - - val labels4 = MesosProtoUtils.mesosLabels("key:value\\,value") - assert(labels4.getLabelsCount == 1) - assert(labels4.getLabels(0).getKey == "key") - assert(labels4.getLabels(0).getValue == "value,value") - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala deleted file mode 100644 index 1fe1ecea4824e..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import org.apache.mesos.Protos.ContainerInfo.DockerInfo - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.mesos.config - -class MesosSchedulerBackendUtilSuite extends SparkFunSuite { - - test("ContainerInfo fails to parse invalid docker parameters") { - val conf = new SparkConf() - conf.set(config.EXECUTOR_DOCKER_PARAMETERS, Seq("a", "b")) - conf.set(config.EXECUTOR_DOCKER_IMAGE, "test") - - val containerInfo = MesosSchedulerBackendUtil.buildContainerInfo( - conf) - val params = containerInfo.getDocker.getParametersList - - assert(params.size() === 0) - } - - test("ContainerInfo parses docker parameters") { - val conf = new SparkConf() - conf.set(config.EXECUTOR_DOCKER_PARAMETERS, Seq("a=1", "b=2", "c=3")) - conf.set(config.EXECUTOR_DOCKER_IMAGE, "test") - - val containerInfo = MesosSchedulerBackendUtil.buildContainerInfo( - conf) - val params = containerInfo.getDocker.getParametersList - assert(params.size() === 3) - assert(params.get(0).getKey === "a") - assert(params.get(0).getValue === "1") - assert(params.get(1).getKey === "b") - assert(params.get(1).getValue === "2") - assert(params.get(2).getKey === "c") - assert(params.get(2).getValue === "3") - } - - test("SPARK-28778 ContainerInfo respects Docker network configuration") { - val networkName = "test" - val conf = new SparkConf() - conf.set(config.CONTAINERIZER, "docker") - conf.set(config.EXECUTOR_DOCKER_IMAGE, "image") - conf.set(config.NETWORK_NAME, networkName) - - val containerInfo = MesosSchedulerBackendUtil.buildContainerInfo(conf) - - assert(containerInfo.getDocker.getNetwork === DockerInfo.Network.USER) - val params = containerInfo.getDocker.getParametersList - assert(params.size() === 1) - assert(params.get(0).getKey === "net") - assert(params.get(0).getValue === networkName) - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala deleted file mode 100644 index f209c607935f8..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala +++ /dev/null @@ -1,398 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.io.{File, FileNotFoundException} -import java.nio.charset.StandardCharsets.UTF_8 - -import scala.jdk.CollectionConverters._ - -import com.google.common.io.Files -import org.apache.mesos.Protos.{FrameworkInfo, Resource, Value} -import org.mockito.Mockito._ -import org.scalatest.matchers.must.Matchers -import org.scalatest.matchers.should.Matchers._ -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} -import org.apache.spark.deploy.mesos.{config => mesosConfig} -import org.apache.spark.internal.config._ -import org.apache.spark.util.SparkConfWithEnv - -class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoSugar { - - class SparkConfFixture { - val sparkConf = new SparkConf - val sc: SparkContext = mock[SparkContext] - when(sc.conf).thenReturn(sparkConf) - } - - def fixture: SparkConfFixture = new SparkConfFixture() - - private def createTestPortResource(range: (Long, Long), role: Option[String] = None): Resource = { - val rangeValue = Value.Range.newBuilder() - rangeValue.setBegin(range._1) - rangeValue.setEnd(range._2) - val builder = Resource.newBuilder() - .setName("ports") - .setType(Value.Type.RANGES) - .setRanges(Value.Ranges.newBuilder().addRange(rangeValue)) - - role.foreach { r => builder.setRole(r) } - builder.build() - } - - private def rangesResourcesToTuple(resources: List[Resource]): List[(Long, Long)] = { - resources.flatMap{resource => resource.getRanges.getRangeList - .asScala.map(range => (range.getBegin, range.getEnd))} - } - - def arePortsEqual(array1: Array[(Long, Long)], array2: Array[(Long, Long)]) - : Boolean = { - array1.sortBy(identity).sameElements(array2.sortBy(identity)) - } - - def arePortsEqual(array1: Array[Long], array2: Array[Long]) - : Boolean = { - array1.sortBy(identity).sameElements(array2.sortBy(identity)) - } - - def getRangesFromResources(resources: List[Resource]): List[(Long, Long)] = { - resources.flatMap{ resource => - resource.getRanges.getRangeList.asScala.toList.map{ - range => (range.getBegin, range.getEnd)}} - } - - val utils = new MesosSchedulerUtils { } - - test("use at-least minimum overhead") { - val f = fixture - when(f.sc.executorMemory).thenReturn(512) - utils.executorMemory(f.sc) shouldBe 896 - } - - test("use overhead if it is greater than minimum value") { - val f = fixture - when(f.sc.executorMemory).thenReturn(4096) - utils.executorMemory(f.sc) shouldBe 4505 - } - - test("use spark.mesos.executor.memoryOverhead (if set)") { - val f = fixture - when(f.sc.executorMemory).thenReturn(1024) - f.sparkConf.set(mesosConfig.EXECUTOR_MEMORY_OVERHEAD, 512) - utils.executorMemory(f.sc) shouldBe 1536 - } - - test("parse a non-empty constraint string correctly") { - val expectedMap = Map( - "os" -> Set("centos7"), - "zone" -> Set("us-east-1a", "us-east-1b") - ) - utils.parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b") should be (expectedMap) - } - - test("parse an empty constraint string correctly") { - utils.parseConstraintString("") shouldBe Map() - } - - test("throw an exception when the input is malformed") { - an[IllegalArgumentException] should be thrownBy - utils.parseConstraintString("os;zone:us-east") - } - - test("empty values for attributes' constraints matches all values") { - val constraintsStr = "os:" - val parsedConstraints = utils.parseConstraintString(constraintsStr) - - parsedConstraints shouldBe Map("os" -> Set()) - - val zoneSet = Value.Set.newBuilder().addItem("us-east-1a").addItem("us-east-1b").build() - val noOsOffer = Map("zone" -> zoneSet) - val centosOffer = Map("os" -> Value.Text.newBuilder().setValue("centos").build()) - val ubuntuOffer = Map("os" -> Value.Text.newBuilder().setValue("ubuntu").build()) - - utils.matchesAttributeRequirements(parsedConstraints, noOsOffer) shouldBe false - utils.matchesAttributeRequirements(parsedConstraints, centosOffer) shouldBe true - utils.matchesAttributeRequirements(parsedConstraints, ubuntuOffer) shouldBe true - } - - test("subset match is performed for set attributes") { - val supersetConstraint = Map( - "os" -> Value.Text.newBuilder().setValue("ubuntu").build(), - "zone" -> Value.Set.newBuilder() - .addItem("us-east-1a") - .addItem("us-east-1b") - .addItem("us-east-1c") - .build()) - - val zoneConstraintStr = "os:;zone:us-east-1a,us-east-1c" - val parsedConstraints = utils.parseConstraintString(zoneConstraintStr) - - utils.matchesAttributeRequirements(parsedConstraints, supersetConstraint) shouldBe true - } - - test("less than equal match is performed on scalar attributes") { - val offerAttribs = Map("gpus" -> Value.Scalar.newBuilder().setValue(3).build()) - - val ltConstraint = utils.parseConstraintString("gpus:2") - val eqConstraint = utils.parseConstraintString("gpus:3") - val gtConstraint = utils.parseConstraintString("gpus:4") - - utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe true - utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true - utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false - } - - test("contains match is performed for range attributes") { - val offerAttribs = Map("ports" -> Value.Range.newBuilder().setBegin(7000).setEnd(8000).build()) - val ltConstraint = utils.parseConstraintString("ports:6000") - val eqConstraint = utils.parseConstraintString("ports:7500") - val gtConstraint = utils.parseConstraintString("ports:8002") - val multiConstraint = utils.parseConstraintString("ports:5000,7500,8300") - - utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe false - utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true - utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false - utils.matchesAttributeRequirements(multiConstraint, offerAttribs) shouldBe true - } - - test("equality match is performed for text attributes") { - val offerAttribs = Map("os" -> Value.Text.newBuilder().setValue("centos7").build()) - - val trueConstraint = utils.parseConstraintString("os:centos7") - val falseConstraint = utils.parseConstraintString("os:ubuntu") - - utils.matchesAttributeRequirements(trueConstraint, offerAttribs) shouldBe true - utils.matchesAttributeRequirements(falseConstraint, offerAttribs) shouldBe false - } - - test("Port reservation is done correctly with user specified ports only") { - val conf = new SparkConf() - conf.set(BLOCK_MANAGER_PORT, 4000) - val portResource = createTestPortResource((3000, 5000), Some("my_role")) - - val (resourcesLeft, resourcesToBeUsed) = utils - .partitionPortResources(List(4000), List(portResource)) - resourcesToBeUsed.length shouldBe 1 - - val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1}.toArray - - portsToUse.length shouldBe 1 - arePortsEqual(portsToUse, Array(4000L)) shouldBe true - - val portRangesToBeUsed = rangesResourcesToTuple(resourcesToBeUsed) - - val expectedUSed = Array((4000L, 4000L)) - - arePortsEqual(portRangesToBeUsed.toArray, expectedUSed) shouldBe true - } - - test("Port reservation is done correctly with all random ports") { - val conf = new SparkConf() - val portResource = createTestPortResource((3000L, 5000L), Some("my_role")) - - val (resourcesLeft, resourcesToBeUsed) = utils - .partitionPortResources(List(), List(portResource)) - val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1} - - portsToUse.isEmpty shouldBe true - } - - test("Port reservation is done correctly with user specified ports only - multiple ranges") { - val conf = new SparkConf() - conf.set(BLOCK_MANAGER_PORT, 4000) - val portResourceList = List(createTestPortResource((3000, 5000), Some("my_role")), - createTestPortResource((2000, 2500), Some("other_role"))) - val (resourcesLeft, resourcesToBeUsed) = utils - .partitionPortResources(List(4000), portResourceList) - val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1} - - portsToUse.length shouldBe 1 - val portsRangesLeft = rangesResourcesToTuple(resourcesLeft) - val portRangesToBeUsed = rangesResourcesToTuple(resourcesToBeUsed) - - val expectedUsed = Array((4000L, 4000L)) - - arePortsEqual(portsToUse.toArray, Array(4000L)) shouldBe true - arePortsEqual(portRangesToBeUsed.toArray, expectedUsed) shouldBe true - } - - test("Port reservation is done correctly with all random ports - multiple ranges") { - val conf = new SparkConf() - val portResourceList = List(createTestPortResource((3000, 5000), Some("my_role")), - createTestPortResource((2000, 2500), Some("other_role"))) - val (resourcesLeft, resourcesToBeUsed) = utils - .partitionPortResources(List(), portResourceList) - val portsToUse = getRangesFromResources(resourcesToBeUsed).map{r => r._1} - portsToUse.isEmpty shouldBe true - } - - test("Principal specified via spark.mesos.principal") { - val conf = new SparkConf() - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - } - - test("Principal specified via spark.mesos.principal.file") { - val pFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt") - pFile.deleteOnExit() - Files.write("test-principal".getBytes(UTF_8), pFile) - val conf = new SparkConf() - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL_FILE, pFile.getAbsolutePath()) - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - } - - test("Principal specified via spark.mesos.principal.file that does not exist") { - val conf = new SparkConf() - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL_FILE, "/tmp/does-not-exist") - - intercept[FileNotFoundException] { - utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - } - } - - test("Principal specified via SPARK_MESOS_PRINCIPAL") { - val conf = new SparkConfWithEnv(Map("SPARK_MESOS_PRINCIPAL" -> "test-principal")) - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - } - - test("Principal specified via SPARK_MESOS_PRINCIPAL_FILE") { - val pFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt") - pFile.deleteOnExit() - Files.write("test-principal".getBytes(UTF_8), pFile) - val conf = new SparkConfWithEnv(Map("SPARK_MESOS_PRINCIPAL_FILE" -> pFile.getAbsolutePath())) - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - } - - test("Principal specified via SPARK_MESOS_PRINCIPAL_FILE that does not exist") { - val conf = new SparkConfWithEnv(Map("SPARK_MESOS_PRINCIPAL_FILE" -> "/tmp/does-not-exist")) - - intercept[FileNotFoundException] { - utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - } - } - - test("Secret specified via spark.mesos.secret") { - val conf = new SparkConf() - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") - conf.set(mesosConfig.CREDENTIAL_SECRET, "my-secret") - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - credBuilder.hasSecret shouldBe true - credBuilder.getSecret shouldBe "my-secret" - } - - test("Principal specified via spark.mesos.secret.file") { - val sFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt") - sFile.deleteOnExit() - Files.write("my-secret".getBytes(UTF_8), sFile) - val conf = new SparkConf() - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") - conf.set(mesosConfig.CREDENTIAL_SECRET_FILE, sFile.getAbsolutePath()) - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - credBuilder.hasSecret shouldBe true - credBuilder.getSecret shouldBe "my-secret" - } - - test("Principal specified via spark.mesos.secret.file that does not exist") { - val conf = new SparkConf() - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") - conf.set(mesosConfig.CREDENTIAL_SECRET_FILE, "/tmp/does-not-exist") - - intercept[FileNotFoundException] { - utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - } - } - - test("Principal specified via SPARK_MESOS_SECRET") { - val env = Map("SPARK_MESOS_SECRET" -> "my-secret") - val conf = new SparkConfWithEnv(env) - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - credBuilder.hasSecret shouldBe true - credBuilder.getSecret shouldBe "my-secret" - } - - test("Principal specified via SPARK_MESOS_SECRET_FILE") { - val sFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt") - sFile.deleteOnExit() - Files.write("my-secret".getBytes(UTF_8), sFile) - - val sFilePath = sFile.getAbsolutePath() - val env = Map("SPARK_MESOS_SECRET_FILE" -> sFilePath) - val conf = new SparkConfWithEnv(env) - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - credBuilder.hasSecret shouldBe true - credBuilder.getSecret shouldBe "my-secret" - } - - test("Secret specified with no principal") { - val conf = new SparkConf() - conf.set(mesosConfig.CREDENTIAL_SECRET, "my-secret") - - intercept[SparkException] { - utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - } - } - - test("Principal specification preference") { - val conf = new SparkConfWithEnv(Map("SPARK_MESOS_PRINCIPAL" -> "other-principal")) - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - } - - test("Secret specification preference") { - val conf = new SparkConfWithEnv(Map("SPARK_MESOS_SECRET" -> "other-secret")) - conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") - conf.set(mesosConfig.CREDENTIAL_SECRET, "my-secret") - - val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) - credBuilder.hasPrincipal shouldBe true - credBuilder.getPrincipal shouldBe "test-principal" - credBuilder.hasSecret shouldBe true - credBuilder.getSecret shouldBe "my-secret" - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala deleted file mode 100644 index ccbdc85931851..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala +++ /dev/null @@ -1,231 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.util.Collections - -import scala.jdk.CollectionConverters._ - -import org.apache.mesos.Protos._ -import org.apache.mesos.Protos.Value.{Range => MesosRange, Ranges, Scalar} -import org.apache.mesos.SchedulerDriver -import org.apache.mesos.protobuf.ByteString -import org.mockito.ArgumentCaptor -import org.mockito.ArgumentMatchers.{any, eq => meq} -import org.mockito.Mockito.{times, verify} -import org.scalatest.Assertions._ - -import org.apache.spark.deploy.mesos.config.MesosSecretConfig - -object Utils { - - val TEST_FRAMEWORK_ID = FrameworkID.newBuilder() - .setValue("test-framework-id") - .build() - - val TEST_MASTER_INFO = MasterInfo.newBuilder() - .setId("test-master") - .setIp(0) - .setPort(0) - .build() - - def createOffer( - offerId: String, - agentId: String, - mem: Int, - cpus: Int, - ports: Option[(Long, Long)] = None, - gpus: Int = 0, - attributes: List[Attribute] = List.empty): Offer = { - val builder = Offer.newBuilder() - builder.addResourcesBuilder() - .setName("mem") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(mem)) - builder.addResourcesBuilder() - .setName("cpus") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(cpus)) - ports.foreach { resourcePorts => - builder.addResourcesBuilder() - .setName("ports") - .setType(Value.Type.RANGES) - .setRanges(Ranges.newBuilder().addRange(MesosRange.newBuilder() - .setBegin(resourcePorts._1).setEnd(resourcePorts._2).build())) - } - if (gpus > 0) { - builder.addResourcesBuilder() - .setName("gpus") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(gpus)) - } - builder.setId(createOfferId(offerId)) - .setFrameworkId(FrameworkID.newBuilder() - .setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(agentId)) - .setHostname(s"host${agentId}") - .addAllAttributes(attributes.asJava) - .build() - } - - def verifyTaskLaunched(driver: SchedulerDriver, offerId: String): List[TaskInfo] = { - val captor = ArgumentCaptor.forClass(classOf[java.util.Collection[TaskInfo]]) - verify(driver, times(1)).launchTasks( - meq(Collections.singleton(createOfferId(offerId))), - captor.capture()) - captor.getValue.asScala.toList - } - - def verifyTaskNotLaunched(driver: SchedulerDriver, offerId: String): Unit = { - verify(driver, times(0)).launchTasks( - meq(Collections.singleton(createOfferId(offerId))), - any(classOf[java.util.Collection[TaskInfo]])) - } - - def createOfferId(offerId: String): OfferID = { - OfferID.newBuilder().setValue(offerId).build() - } - - def createAgentId(agentId: String): SlaveID = { - SlaveID.newBuilder().setValue(agentId).build() - } - - def createExecutorId(executorId: String): ExecutorID = { - ExecutorID.newBuilder().setValue(executorId).build() - } - - def createTaskId(taskId: String): TaskID = { - TaskID.newBuilder().setValue(taskId).build() - } - - def configEnvBasedRefSecrets(secretConfig: MesosSecretConfig): Map[String, String] = { - val secretName = "/path/to/secret,/anothersecret" - val envKey = "SECRET_ENV_KEY,PASSWORD" - Map( - secretConfig.SECRET_NAMES.key -> secretName, - secretConfig.SECRET_ENVKEYS.key -> envKey - ) - } - - def verifyEnvBasedRefSecrets(launchedTasks: List[TaskInfo]): Unit = { - val envVars = launchedTasks.head - .getCommand - .getEnvironment - .getVariablesList - .asScala - assert(envVars.count { x => - !x.getName.startsWith("SPARK_") && x.getName != "OMP_NUM_THREADS" - } == 2) // user-defined secret env vars - val variableOne = envVars.filter(_.getName == "SECRET_ENV_KEY").head - assert(variableOne.getSecret.isInitialized) - assert(variableOne.getSecret.getType == Secret.Type.REFERENCE) - assert(variableOne.getSecret.getReference.getName == "/path/to/secret") - assert(variableOne.getType == Environment.Variable.Type.SECRET) - val variableTwo = envVars.filter(_.getName == "PASSWORD").head - assert(variableTwo.getSecret.isInitialized) - assert(variableTwo.getSecret.getType == Secret.Type.REFERENCE) - assert(variableTwo.getSecret.getReference.getName == "/anothersecret") - assert(variableTwo.getType == Environment.Variable.Type.SECRET) - } - - def configEnvBasedValueSecrets(secretConfig: MesosSecretConfig): Map[String, String] = { - val secretValues = "user,password" - val envKeys = "USER,PASSWORD" - Map( - secretConfig.SECRET_VALUES.key -> secretValues, - secretConfig.SECRET_ENVKEYS.key -> envKeys - ) - } - - def verifyEnvBasedValueSecrets(launchedTasks: List[TaskInfo]): Unit = { - val envVars = launchedTasks.head - .getCommand - .getEnvironment - .getVariablesList - .asScala - assert(envVars.count { x => - !x.getName.startsWith("SPARK_") && x.getName != "OMP_NUM_THREADS" - } == 2) // user-defined secret env vars - val variableOne = envVars.filter(_.getName == "USER").head - assert(variableOne.getSecret.isInitialized) - assert(variableOne.getSecret.getType == Secret.Type.VALUE) - assert(variableOne.getSecret.getValue.getData == - ByteString.copyFrom("user".getBytes)) - assert(variableOne.getType == Environment.Variable.Type.SECRET) - val variableTwo = envVars.filter(_.getName == "PASSWORD").head - assert(variableTwo.getSecret.isInitialized) - assert(variableTwo.getSecret.getType == Secret.Type.VALUE) - assert(variableTwo.getSecret.getValue.getData == - ByteString.copyFrom("password".getBytes)) - assert(variableTwo.getType == Environment.Variable.Type.SECRET) - } - - def configFileBasedRefSecrets(secretConfig: MesosSecretConfig): Map[String, String] = { - val secretName = "/path/to/secret,/anothersecret" - val secretPath = "/topsecret,/mypassword" - Map( - secretConfig.SECRET_NAMES.key -> secretName, - secretConfig.SECRET_FILENAMES.key -> secretPath - ) - } - - def verifyFileBasedRefSecrets(launchedTasks: List[TaskInfo]): Unit = { - val volumes = launchedTasks.head.getContainer.getVolumesList - assert(volumes.size() == 2) - val secretVolOne = volumes.get(0) - assert(secretVolOne.getContainerPath == "/topsecret") - assert(secretVolOne.getSource.getSecret.getType == Secret.Type.REFERENCE) - assert(secretVolOne.getSource.getSecret.getReference.getName == "/path/to/secret") - val secretVolTwo = volumes.get(1) - assert(secretVolTwo.getContainerPath == "/mypassword") - assert(secretVolTwo.getSource.getSecret.getType == Secret.Type.REFERENCE) - assert(secretVolTwo.getSource.getSecret.getReference.getName == "/anothersecret") - } - - def configFileBasedValueSecrets(secretConfig: MesosSecretConfig): Map[String, String] = { - val secretValues = "user,password" - val secretPath = "/whoami,/mypassword" - Map( - secretConfig.SECRET_VALUES.key -> secretValues, - secretConfig.SECRET_FILENAMES.key -> secretPath - ) - } - - def verifyFileBasedValueSecrets(launchedTasks: List[TaskInfo]): Unit = { - val volumes = launchedTasks.head.getContainer.getVolumesList - assert(volumes.size() == 2) - val secretVolOne = volumes.get(0) - assert(secretVolOne.getContainerPath == "/whoami") - assert(secretVolOne.getSource.getSecret.getType == Secret.Type.VALUE) - assert(secretVolOne.getSource.getSecret.getValue.getData == - ByteString.copyFrom("user".getBytes)) - val secretVolTwo = volumes.get(1) - assert(secretVolTwo.getContainerPath == "/mypassword") - assert(secretVolTwo.getSource.getSecret.getType == Secret.Type.VALUE) - assert(secretVolTwo.getSource.getSecret.getValue.getData == - ByteString.copyFrom("password".getBytes)) - } - - def createTextAttribute(name: String, value: String): Attribute = { - Attribute.newBuilder() - .setName(name) - .setType(Value.Type.TEXT) - .setText(Value.Text.newBuilder().setValue(value)) - .build() - } -} diff --git a/sbin/start-mesos-dispatcher.sh b/sbin/start-mesos-dispatcher.sh deleted file mode 100755 index c2e30d8c0b080..0000000000000 --- a/sbin/start-mesos-dispatcher.sh +++ /dev/null @@ -1,51 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# -# Starts the Mesos Cluster Dispatcher on the machine this script is executed on. -# The Mesos Cluster Dispatcher is responsible for launching the Mesos framework and -# Rest server to handle driver requests for Mesos cluster mode. -# Only one cluster dispatcher is needed per Mesos cluster. - -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi - -. "${SPARK_HOME}/sbin/spark-config.sh" - -. "${SPARK_HOME}/bin/load-spark-env.sh" - -if [ "$SPARK_MESOS_DISPATCHER_PORT" = "" ]; then - SPARK_MESOS_DISPATCHER_PORT=7077 -fi - -if [ "$SPARK_MESOS_DISPATCHER_HOST" = "" ]; then - case `uname` in - (SunOS) - SPARK_MESOS_DISPATCHER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" - ;; - (*) - SPARK_MESOS_DISPATCHER_HOST="`hostname -f`" - ;; - esac -fi - -if [ "$SPARK_MESOS_DISPATCHER_NUM" = "" ]; then - SPARK_MESOS_DISPATCHER_NUM=1 -fi - -"${SPARK_HOME}/sbin"/spark-daemon.sh start org.apache.spark.deploy.mesos.MesosClusterDispatcher $SPARK_MESOS_DISPATCHER_NUM --host $SPARK_MESOS_DISPATCHER_HOST --port $SPARK_MESOS_DISPATCHER_PORT "$@" diff --git a/sbin/start-mesos-shuffle-service.sh b/sbin/start-mesos-shuffle-service.sh deleted file mode 100755 index 7b7dc9cf5d866..0000000000000 --- a/sbin/start-mesos-shuffle-service.sh +++ /dev/null @@ -1,36 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# Starts the Mesos external shuffle service on the machine this script is executed on. -# The Mesos external shuffle service detects when an application exits and automatically -# cleans up its shuffle files. -# -# Usage: start-mesos-shuffle-service.sh -# -# Use the SPARK_SHUFFLE_OPTS environment variable to set shuffle service configuration. -# - -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi - -. "${SPARK_HOME}/sbin/spark-config.sh" -. "${SPARK_HOME}/bin/load-spark-env.sh" - -exec "${SPARK_HOME}/sbin"/spark-daemon.sh start org.apache.spark.deploy.mesos.MesosExternalShuffleService 1 diff --git a/sbin/stop-mesos-dispatcher.sh b/sbin/stop-mesos-dispatcher.sh deleted file mode 100755 index b13e018c7d41e..0000000000000 --- a/sbin/stop-mesos-dispatcher.sh +++ /dev/null @@ -1,33 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# -# Stop the Mesos Cluster dispatcher on the machine this script is executed on. - -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi - -. "${SPARK_HOME}/sbin/spark-config.sh" - -if [ "$SPARK_MESOS_DISPATCHER_NUM" = "" ]; then - SPARK_MESOS_DISPATCHER_NUM=1 -fi - -"${SPARK_HOME}/sbin"/spark-daemon.sh stop org.apache.spark.deploy.mesos.MesosClusterDispatcher \ - $SPARK_MESOS_DISPATCHER_NUM - diff --git a/sbin/stop-mesos-shuffle-service.sh b/sbin/stop-mesos-shuffle-service.sh deleted file mode 100755 index d23cad375e1bd..0000000000000 --- a/sbin/stop-mesos-shuffle-service.sh +++ /dev/null @@ -1,26 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# Stops the Mesos external shuffle service on the machine this script is executed on. - -if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -fi - -"${SPARK_HOME}/sbin"/spark-daemon.sh stop org.apache.spark.deploy.mesos.MesosExternalShuffleService 1 diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index be503084b2a34..1500f72763f9f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -64,8 +64,7 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) "spark.yarn.principal", "spark.kerberos.keytab", "spark.kerberos.principal", - UI_FILTERS.key, - "spark.mesos.driver.frameworkId") + UI_FILTERS.key) val newSparkConf = new SparkConf(loadDefaults = false).setAll(sparkConfPairs) .remove("spark.driver.host") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index afeca67956b41..30bd30329283b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -93,7 +93,7 @@ class StreamingContext private[streaming] ( /** * Create a StreamingContext by providing the details necessary for creating a new SparkContext. - * @param master cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param master cluster URL to connect to (e.g. spark://host:port, local[4]). * @param appName a name for your job, to display on the cluster web UI * @param batchDuration the time interval at which streaming data will be divided into batches */