Skip to content

Commit 8e5475b

Browse files
Michael GummeltMarcelo Vanzin
Michael Gummelt
authored and
Marcelo Vanzin
committed
[SPARK-16967] move mesos to module
## What changes were proposed in this pull request? Move Mesos code into a mvn module ## How was this patch tested? unit tests manually submitting a client mode and cluster mode job spark/mesos integration test suite Author: Michael Gummelt <[email protected]> Closes apache#14637 from mgummelt/mesos-module.
1 parent c0949dc commit 8e5475b

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

43 files changed

+305
-118
lines changed

.travis.yml

+1-1
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@ notifications:
4444
# 5. Run maven install before running lint-java.
4545
install:
4646
- export MAVEN_SKIP_RC=1
47-
- build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
47+
- build/mvn -T 4 -q -DskipTests -Pmesos -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
4848

4949
# 6. Run lint-java.
5050
script:

assembly/pom.xml

+10
Original file line numberDiff line numberDiff line change
@@ -138,6 +138,16 @@
138138
</dependency>
139139
</dependencies>
140140
</profile>
141+
<profile>
142+
<id>mesos</id>
143+
<dependencies>
144+
<dependency>
145+
<groupId>org.apache.spark</groupId>
146+
<artifactId>spark-mesos_${scala.binary.version}</artifactId>
147+
<version>${project.version}</version>
148+
</dependency>
149+
</dependencies>
150+
</profile>
141151
<profile>
142152
<id>hive</id>
143153
<dependencies>

core/pom.xml

-5
Original file line numberDiff line numberDiff line change
@@ -215,11 +215,6 @@
215215
<groupId>org.glassfish.jersey.containers</groupId>
216216
<artifactId>jersey-container-servlet-core</artifactId>
217217
</dependency>
218-
<dependency>
219-
<groupId>org.apache.mesos</groupId>
220-
<artifactId>mesos</artifactId>
221-
<classifier>${mesos.classifier}</classifier>
222-
</dependency>
223218
<dependency>
224219
<groupId>io.netty</groupId>
225220
<artifactId>netty-all</artifactId>

core/src/main/scala/org/apache/spark/SparkContext.scala

+1-17
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,6 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, Sequence
4242
TextInputFormat}
4343
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob}
4444
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
45-
import org.apache.mesos.MesosNativeLibrary
4645

4746
import org.apache.spark.annotation.DeveloperApi
4847
import org.apache.spark.broadcast.Broadcast
@@ -56,7 +55,6 @@ import org.apache.spark.rdd._
5655
import org.apache.spark.rpc.RpcEndpointRef
5756
import org.apache.spark.scheduler._
5857
import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend}
59-
import org.apache.spark.scheduler.cluster.mesos.{MesosCoarseGrainedSchedulerBackend, MesosFineGrainedSchedulerBackend}
6058
import org.apache.spark.scheduler.local.LocalSchedulerBackend
6159
import org.apache.spark.storage._
6260
import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump
@@ -2512,18 +2510,6 @@ object SparkContext extends Logging {
25122510
}
25132511
(backend, scheduler)
25142512

2515-
case MESOS_REGEX(mesosUrl) =>
2516-
MesosNativeLibrary.load()
2517-
val scheduler = new TaskSchedulerImpl(sc)
2518-
val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", defaultValue = true)
2519-
val backend = if (coarseGrained) {
2520-
new MesosCoarseGrainedSchedulerBackend(scheduler, sc, mesosUrl, sc.env.securityManager)
2521-
} else {
2522-
new MesosFineGrainedSchedulerBackend(scheduler, sc, mesosUrl)
2523-
}
2524-
scheduler.initialize(backend)
2525-
(backend, scheduler)
2526-
25272513
case masterUrl =>
25282514
val cm = getClusterManager(masterUrl) match {
25292515
case Some(clusterMgr) => clusterMgr
@@ -2545,7 +2531,7 @@ object SparkContext extends Logging {
25452531
private def getClusterManager(url: String): Option[ExternalClusterManager] = {
25462532
val loader = Utils.getContextOrSparkClassLoader
25472533
val serviceLoaders =
2548-
ServiceLoader.load(classOf[ExternalClusterManager], loader).asScala.filter(_.canCreate(url))
2534+
ServiceLoader.load(classOf[ExternalClusterManager], loader).asScala.filter(_.canCreate(url))
25492535
if (serviceLoaders.size > 1) {
25502536
throw new SparkException(s"Multiple Cluster Managers ($serviceLoaders) registered " +
25512537
s"for the url $url:")
@@ -2566,8 +2552,6 @@ private object SparkMasterRegex {
25662552
val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r
25672553
// Regular expression for connecting to Spark deploy clusters
25682554
val SPARK_REGEX = """spark://(.*)""".r
2569-
// Regular expression for connection to Mesos cluster by mesos:// or mesos://zk:// url
2570-
val MESOS_REGEX = """mesos://(.*)""".r
25712555
}
25722556

25732557
/**

core/src/main/scala/org/apache/spark/TaskState.scala

-20
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,6 @@
1717

1818
package org.apache.spark
1919

20-
import org.apache.mesos.Protos.{TaskState => MesosTaskState}
21-
2220
private[spark] object TaskState extends Enumeration {
2321

2422
val LAUNCHING, RUNNING, FINISHED, FAILED, KILLED, LOST = Value
@@ -30,22 +28,4 @@ private[spark] object TaskState extends Enumeration {
3028
def isFailed(state: TaskState): Boolean = (LOST == state) || (FAILED == state)
3129

3230
def isFinished(state: TaskState): Boolean = FINISHED_STATES.contains(state)
33-
34-
def toMesos(state: TaskState): MesosTaskState = state match {
35-
case LAUNCHING => MesosTaskState.TASK_STARTING
36-
case RUNNING => MesosTaskState.TASK_RUNNING
37-
case FINISHED => MesosTaskState.TASK_FINISHED
38-
case FAILED => MesosTaskState.TASK_FAILED
39-
case KILLED => MesosTaskState.TASK_KILLED
40-
case LOST => MesosTaskState.TASK_LOST
41-
}
42-
43-
def fromMesos(mesosState: MesosTaskState): TaskState = mesosState match {
44-
case MesosTaskState.TASK_STAGING | MesosTaskState.TASK_STARTING => LAUNCHING
45-
case MesosTaskState.TASK_RUNNING | MesosTaskState.TASK_KILLING => RUNNING
46-
case MesosTaskState.TASK_FINISHED => FINISHED
47-
case MesosTaskState.TASK_FAILED => FAILED
48-
case MesosTaskState.TASK_KILLED => KILLED
49-
case MesosTaskState.TASK_LOST | MesosTaskState.TASK_ERROR => LOST
50-
}
5131
}

core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala

-28
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@ import org.scalatest.PrivateMethodTester
2222
import org.apache.spark.internal.Logging
2323
import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
2424
import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend
25-
import org.apache.spark.scheduler.cluster.mesos.{MesosCoarseGrainedSchedulerBackend, MesosFineGrainedSchedulerBackend}
2625
import org.apache.spark.scheduler.local.LocalSchedulerBackend
2726

2827

@@ -130,31 +129,4 @@ class SparkContextSchedulerCreationSuite
130129
case _ => fail()
131130
}
132131
}
133-
134-
def testMesos(master: String, expectedClass: Class[_], coarse: Boolean) {
135-
val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString)
136-
try {
137-
val sched = createTaskScheduler(master, "client", conf)
138-
assert(sched.backend.getClass === expectedClass)
139-
} catch {
140-
case e: UnsatisfiedLinkError =>
141-
assert(e.getMessage.contains("mesos"))
142-
logWarning("Mesos not available, could not test actual Mesos scheduler creation")
143-
case e: Throwable => fail(e)
144-
}
145-
}
146-
147-
test("mesos fine-grained") {
148-
testMesos("mesos://localhost:1234", classOf[MesosFineGrainedSchedulerBackend], coarse = false)
149-
}
150-
151-
test("mesos coarse-grained") {
152-
testMesos("mesos://localhost:1234", classOf[MesosCoarseGrainedSchedulerBackend], coarse = true)
153-
}
154-
155-
test("mesos with zookeeper") {
156-
testMesos("mesos://zk://localhost:1234,localhost:2345",
157-
classOf[MesosFineGrainedSchedulerBackend], coarse = false)
158-
}
159-
160132
}

dev/create-release/release-build.sh

+8-7
Original file line numberDiff line numberDiff line change
@@ -80,7 +80,7 @@ NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads
8080
BASE_DIR=$(pwd)
8181

8282
MVN="build/mvn --force"
83-
PUBLISH_PROFILES="-Pyarn -Phive -Phive-thriftserver -Phadoop-2.2"
83+
PUBLISH_PROFILES="-Pmesos -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2"
8484
PUBLISH_PROFILES="$PUBLISH_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl"
8585

8686
rm -rf spark
@@ -186,12 +186,13 @@ if [[ "$1" == "package" ]]; then
186186

187187
# We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds
188188
# share the same Zinc server.
189-
make_binary_release "hadoop2.3" "-Psparkr -Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" "3033" &
190-
make_binary_release "hadoop2.4" "-Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" "3034" &
191-
make_binary_release "hadoop2.6" "-Psparkr -Phadoop-2.6 -Phive -Phive-thriftserver -Pyarn" "3035" &
192-
make_binary_release "hadoop2.7" "-Psparkr -Phadoop-2.7 -Phive -Phive-thriftserver -Pyarn" "3036" &
193-
make_binary_release "hadoop2.4-without-hive" "-Psparkr -Phadoop-2.4 -Pyarn" "3037" &
194-
make_binary_release "without-hadoop" "-Psparkr -Phadoop-provided -Pyarn" "3038" &
189+
FLAGS="-Psparkr -Phive -Phive-thriftserver -Pyarn -Pmesos"
190+
make_binary_release "hadoop2.3" "-Phadoop2.3 $FLAGS" "3033" &
191+
make_binary_release "hadoop2.4" "-Phadoop2.4 $FLAGS" "3034" &
192+
make_binary_release "hadoop2.6" "-Phadoop2.6 $FLAGS" "3035" &
193+
make_binary_release "hadoop2.7" "-Phadoop2.7 $FLAGS" "3036" &
194+
make_binary_release "hadoop2.4-without-hive" "-Psparkr -Phadoop-2.4 -Pyarn -Pmesos" "3037" &
195+
make_binary_release "without-hadoop" "-Psparkr -Phadoop-provided -Pyarn -Pmesos" "3038" &
195196
wait
196197
rm -rf spark-$SPARK_VERSION-bin-*/
197198

dev/lint-java

+1-1
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@
2020
SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )"
2121
SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)"
2222

23-
ERRORS=$($SCRIPT_DIR/../build/mvn -Pkinesis-asl -Pyarn -Phive -Phive-thriftserver checkstyle:check | grep ERROR)
23+
ERRORS=$($SCRIPT_DIR/../build/mvn -Pkinesis-asl -Pmesos -Pyarn -Phive -Phive-thriftserver checkstyle:check | grep ERROR)
2424

2525
if test ! -z "$ERRORS"; then
2626
echo -e "Checkstyle checks failed at following occurrences:\n$ERRORS"

dev/mima

+1-1
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ set -e
2424
FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
2525
cd "$FWDIR"
2626

27-
SPARK_PROFILES="-Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"
27+
SPARK_PROFILES="-Pmesos -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"
2828
TOOLS_CLASSPATH="$(build/sbt -DcopyDependencies=false "export tools/fullClasspath" | tail -n1)"
2929
OLD_DEPS_CLASSPATH="$(build/sbt -DcopyDependencies=false $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)"
3030

dev/scalastyle

+1
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
ERRORS=$(echo -e "q\n" \
2323
| build/sbt \
2424
-Pkinesis-asl \
25+
-Pmesos \
2526
-Pyarn \
2627
-Phive \
2728
-Phive-thriftserver \

dev/sparktestsupport/modules.py

+7
Original file line numberDiff line numberDiff line change
@@ -458,6 +458,13 @@ def __hash__(self):
458458
]
459459
)
460460

461+
mesos = Module(
462+
name="mesos",
463+
dependencies=[],
464+
source_file_regexes=["mesos/"],
465+
sbt_test_goals=["mesos/test"]
466+
)
467+
461468
# The root module is a dummy module which is used to run all of the tests.
462469
# No other modules should directly depend on this module.
463470
root = Module(

dev/test-dependencies.sh

+1-1
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ export LC_ALL=C
2929
# TODO: This would be much nicer to do in SBT, once SBT supports Maven-style resolution.
3030

3131
# NOTE: These should match those in the release publishing script
32-
HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pyarn -Phive"
32+
HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pyarn -Phive"
3333
MVN="build/mvn"
3434
HADOOP_PROFILES=(
3535
hadoop-2.2

docs/building-spark.md

+14-10
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ To create a Spark distribution like those distributed by the
5050
to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured
5151
with Maven profile settings and so on like the direct Maven build. Example:
5252

53-
./dev/make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn
53+
./dev/make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pmesos -Pyarn
5454

5555
For more information on usage, run `./dev/make-distribution.sh --help`
5656

@@ -105,13 +105,17 @@ By default Spark will build with Hive 1.2.1 bindings.
105105

106106
## Packaging without Hadoop Dependencies for YARN
107107

108-
The assembly directory produced by `mvn package` will, by default, include all of Spark's
109-
dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this
110-
causes multiple versions of these to appear on executor classpaths: the version packaged in
108+
The assembly directory produced by `mvn package` will, by default, include all of Spark's
109+
dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this
110+
causes multiple versions of these to appear on executor classpaths: the version packaged in
111111
the Spark assembly and the version on each node, included with `yarn.application.classpath`.
112-
The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects,
112+
The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects,
113113
like ZooKeeper and Hadoop itself.
114114

115+
## Building with Mesos support
116+
117+
./build/mvn -Pmesos -DskipTests clean package
118+
115119
## Building for Scala 2.10
116120
To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property:
117121

@@ -263,17 +267,17 @@ The run-tests script also can be limited to a specific Python version or a speci
263267

264268
## Running R Tests
265269

266-
To run the SparkR tests you will need to install the R package `testthat`
267-
(run `install.packages(testthat)` from R shell). You can run just the SparkR tests using
270+
To run the SparkR tests you will need to install the R package `testthat`
271+
(run `install.packages(testthat)` from R shell). You can run just the SparkR tests using
268272
the command:
269273

270274
./R/run-tests.sh
271275

272276
## Running Docker-based Integration Test Suites
273277

274-
In order to run Docker integration tests, you have to install the `docker` engine on your box.
275-
The instructions for installation can be found at [the Docker site](https://docs.docker.com/engine/installation/).
276-
Once installed, the `docker` service needs to be started, if not already running.
278+
In order to run Docker integration tests, you have to install the `docker` engine on your box.
279+
The instructions for installation can be found at [the Docker site](https://docs.docker.com/engine/installation/).
280+
Once installed, the `docker` service needs to be started, if not already running.
277281
On Linux, this can be done by `sudo service docker start`.
278282

279283
./build/mvn install -DskipTests

0 commit comments

Comments
 (0)