Merging upstream changes.

This commit is contained in:
Joseph E. Gonzalez 2013-08-27 18:26:54 -07:00
commit 9afd0e2375
128 changed files with 3059 additions and 3036 deletions

View file

@ -16,7 +16,7 @@ Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The project is
built using Simple Build Tool (SBT), which is packaged with it. To build
Spark and its example programs, run:
sbt/sbt package
sbt/sbt package assembly
Spark also supports building using Maven. If you would like to build using Maven,
see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html)
@ -43,10 +43,47 @@ locally with one thread, or "local[N]" to run locally with N threads.
## A Note About Hadoop Versions
Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported
storage systems. Because the HDFS API has changed in different versions of
storage systems. Because the protocols have changed in different versions of
Hadoop, you must build Spark against the same version that your cluster runs.
You can change the version by setting the `HADOOP_VERSION` variable at the top
of `project/SparkBuild.scala`, then rebuilding Spark.
You can change the version by setting the `SPARK_HADOOP_VERSION` environment
when building Spark.
For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop
versions without YARN, use:
# Apache Hadoop 1.2.1
$ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt package assembly
# Cloudera CDH 4.2.0 with MapReduce v1
$ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt package assembly
For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions
with YARN, also set `SPARK_WITH_YARN=true`:
# Apache Hadoop 2.0.5-alpha
$ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true sbt/sbt package assembly
# Cloudera CDH 4.2.0 with MapReduce v2
$ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_WITH_YARN=true sbt/sbt package assembly
For convenience, these variables may also be set through the `conf/spark-env.sh` file
described below.
When developing a Spark application, specify the Hadoop version by adding the
"hadoop-client" artifact to your project's dependencies. For example, if you're
using Hadoop 1.0.1 and build your application using SBT, add this entry to
`libraryDependencies`:
"org.apache.hadoop" % "hadoop-client" % "1.2.1"
If your project is built with Maven, add this to your POM file's `<dependencies>` section:
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<!-- the brackets are needed to tell Maven that this is a hard dependency on version "1.2.1" exactly -->
<version>[1.2.1]</version>
</dependency>
## Configuration

View file

@ -37,55 +37,30 @@
</plugins>
</build>
<profiles>
<profile>
<id>hadoop1</id>
<properties>
<classifier.name>hadoop1</classifier.name>
</properties>
</profile>
<profile>
<id>hadoop2</id>
<properties>
<classifier.name>hadoop2</classifier.name>
</properties>
</profile>
<profile>
<id>hadoop2-yarn</id>
<properties>
<classifier.name>hadoop2-yarn</classifier.name>
</properties>
</profile>
</profiles>
<dependencies>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-core</artifactId>
<classifier>${classifier.name}</classifier>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-bagel</artifactId>
<classifier>${classifier.name}</classifier>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-mllib</artifactId>
<classifier>${classifier.name}</classifier>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-repl</artifactId>
<classifier>${classifier.name}</classifier>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-streaming</artifactId>
<classifier>${classifier.name}</classifier>
<version>${project.version}</version>
</dependency>
</dependencies>

View file

@ -32,11 +32,15 @@
<url>http://spark-project.org/</url>
<dependencies>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-core</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.version}</artifactId>
@ -58,103 +62,4 @@
</plugin>
</plugins>
</build>
<profiles>
<profile>
<id>hadoop1</id>
<dependencies>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-core</artifactId>
<version>${project.version}</version>
<classifier>hadoop1</classifier>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<classifier>hadoop1</classifier>
</configuration>
</plugin>
</plugins>
</build>
</profile>
<profile>
<id>hadoop2</id>
<dependencies>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-core</artifactId>
<version>${project.version}</version>
<classifier>hadoop2</classifier>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<classifier>hadoop2</classifier>
</configuration>
</plugin>
</plugins>
</build>
</profile>
<profile>
<id>hadoop2-yarn</id>
<dependencies>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-core</artifactId>
<version>${project.version}</version>
<classifier>hadoop2-yarn</classifier>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-api</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-common</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<classifier>hadoop2-yarn</classifier>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View file

@ -34,6 +34,7 @@ set EXAMPLES_DIR=%FWDIR%examples
set BAGEL_DIR=%FWDIR%bagel
set MLLIB_DIR=%FWDIR%mllib
set TOOLS_DIR=%FWDIR%tools
set YARN_DIR=%FWDIR%yarn
set STREAMING_DIR=%FWDIR%streaming
set PYSPARK_DIR=%FWDIR%python
@ -50,6 +51,7 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\*
set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes
set CLASSPATH=%CLASSPATH%;%MLLIB_DIR%\target\scala-%SCALA_VERSION%\classes
set CLASSPATH=%CLASSPATH%;%TOOLS_DIR%\target\scala-%SCALA_VERSION%\classes
set CLASSPATH=%CLASSPATH%;%YARN_DIR%\target\scala-%SCALA_VERSION%\classes
rem Add hadoop conf dir - else FileSystem.*, etc fail
rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts

View file

@ -37,6 +37,7 @@ EXAMPLES_DIR="$FWDIR/examples"
BAGEL_DIR="$FWDIR/bagel"
MLLIB_DIR="$FWDIR/mllib"
TOOLS_DIR="$FWDIR/tools"
YARN_DIR="$FWDIR/yarn"
STREAMING_DIR="$FWDIR/streaming"
PYSPARK_DIR="$FWDIR/python"
@ -62,16 +63,18 @@ function dev_classpath {
CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*"
# Add the shaded JAR for Maven builds
if [ -e $REPL_BIN_DIR/target ]; then
for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded.jar'`; do
CLASSPATH="$CLASSPATH:$jar"
done
# The shaded JAR doesn't contain examples, so include those separately
EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
CLASSPATH+=":$EXAMPLES_JAR"
for jar in `find "$EXAMPLES_DIR/target" -name 'spark-examples*[0-9T].jar'`; do
CLASSPATH="$CLASSPATH:$jar"
done
fi
CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$TOOLS_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH="$CLASSPATH:$YARN_DIR/target/scala-$SCALA_VERSION/classes"
for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
CLASSPATH="$CLASSPATH:$jar"
done

View file

@ -32,6 +32,18 @@
<url>http://spark-project.org/</url>
<dependencies>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-ipc</artifactId>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
@ -40,6 +52,10 @@
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
@ -90,6 +106,10 @@
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
</dependency>
<dependency>
<groupId>net.liftweb</groupId>
<artifactId>lift-json_2.9.2</artifactId>
</dependency>
<dependency>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
@ -126,7 +146,6 @@
<groupId>com.codahale.metrics</groupId>
<artifactId>metrics-json</artifactId>
</dependency>
<dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derby</artifactId>
@ -204,183 +223,4 @@
</plugin>
</plugins>
</build>
<profiles>
<profile>
<id>hadoop1</id>
<dependencies>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<executions>
<execution>
<id>add-source</id>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/scala</source>
<source>src/hadoop1/scala</source>
</sources>
</configuration>
</execution>
<execution>
<id>add-scala-test-sources</id>
<phase>generate-test-sources</phase>
<goals>
<goal>add-test-source</goal>
</goals>
<configuration>
<sources>
<source>src/test/scala</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<classifier>hadoop1</classifier>
</configuration>
</plugin>
</plugins>
</build>
</profile>
<profile>
<id>hadoop2</id>
<dependencies>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<executions>
<execution>
<id>add-source</id>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/scala</source>
<source>src/hadoop2/scala</source>
</sources>
</configuration>
</execution>
<execution>
<id>add-scala-test-sources</id>
<phase>generate-test-sources</phase>
<goals>
<goal>add-test-source</goal>
</goals>
<configuration>
<sources>
<source>src/test/scala</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<classifier>hadoop2</classifier>
</configuration>
</plugin>
</plugins>
</build>
</profile>
<profile>
<id>hadoop2-yarn</id>
<dependencies>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-api</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-common</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-client</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<executions>
<execution>
<id>add-source</id>
<phase>generate-sources</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/scala</source>
<source>src/hadoop2-yarn/scala</source>
</sources>
</configuration>
</execution>
<execution>
<id>add-scala-test-sources</id>
<phase>generate-test-sources</phase>
<goals>
<goal>add-test-source</goal>
</goals>
<configuration>
<sources>
<source>src/test/scala</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<classifier>hadoop2-yarn</classifier>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View file

@ -1,47 +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 spark.deploy
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapred.JobConf
/**
* Contains util methods to interact with Hadoop from spark.
*/
object SparkHadoopUtil {
def getUserNameFromEnvironment(): String = {
// defaulting to -D ...
System.getProperty("user.name")
}
def runAsUser(func: (Product) => Unit, args: Product) {
// Add support, if exists - for now, simply run func !
func(args)
}
// Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
def newConfiguration(): Configuration = new Configuration()
// add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster
def addCredentials(conf: JobConf) {}
def isYarnMode(): Boolean = { false }
}

View file

@ -1,30 +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.hadoop.mapreduce
import org.apache.hadoop.conf.Configuration
import task.{TaskAttemptContextImpl, JobContextImpl}
trait HadoopMapReduceUtil {
def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier,
jobId, isMap, taskId, attemptId)
}

View file

@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred
trait SparkHadoopMapRedUtil {
def newJobContext(conf: JobConf, jobId: JobID): JobContext = {
val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", "org.apache.hadoop.mapred.JobContext");
val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[org.apache.hadoop.mapreduce.JobID])
ctor.newInstance(conf, jobId).asInstanceOf[JobContext]
}
def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = {
val klass = firstAvailableClass("org.apache.hadoop.mapred.TaskAttemptContextImpl", "org.apache.hadoop.mapred.TaskAttemptContext")
val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[TaskAttemptID])
ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext]
}
def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = {
new TaskAttemptID(jtIdentifier, jobId, isMap, taskId, attemptId)
}
private def firstAvailableClass(first: String, second: String): Class[_] = {
try {
Class.forName(first)
} catch {
case e: ClassNotFoundException =>
Class.forName(second)
}
}
}

View file

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce
import org.apache.hadoop.conf.Configuration
import java.lang.{Integer => JInteger, Boolean => JBoolean}
trait SparkHadoopMapReduceUtil {
def newJobContext(conf: Configuration, jobId: JobID): JobContext = {
val klass = firstAvailableClass(
"org.apache.hadoop.mapreduce.task.JobContextImpl", // hadoop2, hadoop2-yarn
"org.apache.hadoop.mapreduce.JobContext") // hadoop1
val ctor = klass.getDeclaredConstructor(classOf[Configuration], classOf[JobID])
ctor.newInstance(conf, jobId).asInstanceOf[JobContext]
}
def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = {
val klass = firstAvailableClass(
"org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl", // hadoop2, hadoop2-yarn
"org.apache.hadoop.mapreduce.TaskAttemptContext") // hadoop1
val ctor = klass.getDeclaredConstructor(classOf[Configuration], classOf[TaskAttemptID])
ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext]
}
def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = {
val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID");
try {
// first, attempt to use the old-style constructor that takes a boolean isMap (not available in YARN)
val ctor = klass.getDeclaredConstructor(classOf[String], classOf[Int], classOf[Boolean],
classOf[Int], classOf[Int])
ctor.newInstance(jtIdentifier, new JInteger(jobId), new JBoolean(isMap), new JInteger(taskId), new
JInteger(attemptId)).asInstanceOf[TaskAttemptID]
} catch {
case exc: NoSuchMethodException => {
// failed, look for the new ctor that takes a TaskType (not available in 1.x)
val taskTypeClass = Class.forName("org.apache.hadoop.mapreduce.TaskType").asInstanceOf[Class[Enum[_]]]
val taskType = taskTypeClass.getMethod("valueOf", classOf[String]).invoke(taskTypeClass, if(isMap) "MAP" else "REDUCE")
val ctor = klass.getDeclaredConstructor(classOf[String], classOf[Int], taskTypeClass,
classOf[Int], classOf[Int])
ctor.newInstance(jtIdentifier, new JInteger(jobId), taskType, new JInteger(taskId), new
JInteger(attemptId)).asInstanceOf[TaskAttemptID]
}
}
}
private def firstAvailableClass(first: String, second: String): Class[_] = {
try {
Class.forName(first)
} catch {
case e: ClassNotFoundException =>
Class.forName(second)
}
}
}

View file

@ -28,18 +28,18 @@ import scala.collection.JavaConversions._
* @param mergeCombiners function to merge outputs from multiple mergeValue function.
*/
case class Aggregator[K, V, C] (
val createCombiner: V => C,
val mergeValue: (C, V) => C,
val mergeCombiners: (C, C) => C) {
createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiners: (C, C) => C) {
def combineValuesByKey(iter: Iterator[(K, V)]) : Iterator[(K, C)] = {
def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = {
val combiners = new JHashMap[K, C]
for ((k, v) <- iter) {
val oldC = combiners.get(k)
for (kv <- iter) {
val oldC = combiners.get(kv._1)
if (oldC == null) {
combiners.put(k, createCombiner(v))
combiners.put(kv._1, createCombiner(kv._2))
} else {
combiners.put(k, mergeValue(oldC, v))
combiners.put(kv._1, mergeValue(oldC, kv._2))
}
}
combiners.iterator
@ -47,7 +47,7 @@ case class Aggregator[K, V, C] (
def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = {
val combiners = new JHashMap[K, C]
for ((k, c) <- iter) {
iter.foreach { case(k, c) =>
val oldC = combiners.get(k)
if (oldC == null) {
combiners.put(k, c)

View file

@ -28,8 +28,9 @@ import spark.util.CompletionIterator
private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
override def fetch[K, V](
shuffleId: Int, reduceId: Int, metrics: TaskMetrics, serializer: Serializer) = {
override def fetch[T](shuffleId: Int, reduceId: Int, metrics: TaskMetrics, serializer: Serializer)
: Iterator[T] =
{
logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId))
val blockManager = SparkEnv.get.blockManager
@ -49,12 +50,12 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
(address, splits.map(s => ("shuffle_%d_%d_%d".format(shuffleId, s._1, reduceId), s._2)))
}
def unpackBlock(blockPair: (String, Option[Iterator[Any]])) : Iterator[(K, V)] = {
def unpackBlock(blockPair: (String, Option[Iterator[Any]])) : Iterator[T] = {
val blockId = blockPair._1
val blockOption = blockPair._2
blockOption match {
case Some(block) => {
block.asInstanceOf[Iterator[(K, V)]]
block.asInstanceOf[Iterator[T]]
}
case None => {
val regex = "shuffle_([0-9]*)_([0-9]*)_([0-9]*)".r
@ -73,7 +74,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer)
val itr = blockFetcherItr.flatMap(unpackBlock)
CompletionIterator[(K,V), Iterator[(K,V)]](itr, {
CompletionIterator[T, Iterator[T]](itr, {
val shuffleMetrics = new ShuffleReadMetrics
shuffleMetrics.shuffleFinishTime = System.currentTimeMillis
shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime

View file

@ -44,10 +44,10 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
* @param serializerClass class name of the serializer to use
*/
class ShuffleDependency[K, V](
@transient rdd: RDD[(K, V)],
@transient rdd: RDD[_ <: Product2[K, V]],
val partitioner: Partitioner,
val serializerClass: String = null)
extends Dependency(rdd) {
extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) {
val shuffleId: Int = rdd.context.newShuffleId()
}

View file

@ -209,7 +209,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K
if (other.partitioner == Some(partitioner)) {
other
} else {
new ShuffledRDD[K,W](other, partitioner)
new ShuffledRDD[K, W, (K,W)](other, partitioner)
}
// Join the other RDD with this RDD building a new valueset and new index on the fly
val groups =

View file

@ -64,11 +64,11 @@ private[spark] class MapOutputTracker extends Logging {
// Incremented every time a fetch fails so that client nodes know to clear
// their cache of map output locations if this happens.
private var generation: Long = 0
private val generationLock = new java.lang.Object
private var epoch: Long = 0
private val epochLock = new java.lang.Object
// Cache a serialized version of the output statuses for each shuffle to send them out faster
var cacheGeneration = generation
var cacheEpoch = epoch
private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]
val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup)
@ -108,10 +108,10 @@ private[spark] class MapOutputTracker extends Logging {
def registerMapOutputs(
shuffleId: Int,
statuses: Array[MapStatus],
changeGeneration: Boolean = false) {
changeEpoch: Boolean = false) {
mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses)
if (changeGeneration) {
incrementGeneration()
if (changeEpoch) {
incrementEpoch()
}
}
@ -124,7 +124,7 @@ private[spark] class MapOutputTracker extends Logging {
array(mapId) = null
}
}
incrementGeneration()
incrementEpoch()
} else {
throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID")
}
@ -206,58 +206,58 @@ private[spark] class MapOutputTracker extends Logging {
trackerActor = null
}
// Called on master to increment the generation number
def incrementGeneration() {
generationLock.synchronized {
generation += 1
logDebug("Increasing generation to " + generation)
// Called on master to increment the epoch number
def incrementEpoch() {
epochLock.synchronized {
epoch += 1
logDebug("Increasing epoch to " + epoch)
}
}
// Called on master or workers to get current generation number
def getGeneration: Long = {
generationLock.synchronized {
return generation
// Called on master or workers to get current epoch number
def getEpoch: Long = {
epochLock.synchronized {
return epoch
}
}
// Called on workers to update the generation number, potentially clearing old outputs
// because of a fetch failure. (Each Mesos task calls this with the latest generation
// Called on workers to update the epoch number, potentially clearing old outputs
// because of a fetch failure. (Each worker task calls this with the latest epoch
// number on the master at the time it was created.)
def updateGeneration(newGen: Long) {
generationLock.synchronized {
if (newGen > generation) {
logInfo("Updating generation to " + newGen + " and clearing cache")
def updateEpoch(newEpoch: Long) {
epochLock.synchronized {
if (newEpoch > epoch) {
logInfo("Updating epoch to " + newEpoch + " and clearing cache")
// mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
mapStatuses.clear()
generation = newGen
epoch = newEpoch
}
}
}
def getSerializedLocations(shuffleId: Int): Array[Byte] = {
var statuses: Array[MapStatus] = null
var generationGotten: Long = -1
generationLock.synchronized {
if (generation > cacheGeneration) {
var epochGotten: Long = -1
epochLock.synchronized {
if (epoch > cacheEpoch) {
cachedSerializedStatuses.clear()
cacheGeneration = generation
cacheEpoch = epoch
}
cachedSerializedStatuses.get(shuffleId) match {
case Some(bytes) =>
return bytes
case None =>
statuses = mapStatuses(shuffleId)
generationGotten = generation
epochGotten = epoch
}
}
// If we got here, we failed to find the serialized locations in the cache, so we pulled
// out a snapshot of the locations as "locs"; let's serialize and return that
val bytes = serializeStatuses(statuses)
logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length))
// Add them into the table only if the generation hasn't changed while we were working
generationLock.synchronized {
if (generation == generationGotten) {
// Add them into the table only if the epoch hasn't changed while we were working
epochLock.synchronized {
if (epoch == epochGotten) {
cachedSerializedStatuses(shuffleId) = bytes
}
}

View file

@ -21,9 +21,8 @@ import java.nio.ByteBuffer
import java.util.{Date, HashMap => JHashMap}
import java.text.SimpleDateFormat
import scala.collection.Map
import scala.collection.{mutable, Map}
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
import scala.collection.JavaConversions._
import org.apache.hadoop.conf.Configuration
@ -32,12 +31,13 @@ import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.io.SequenceFile.CompressionType
import org.apache.hadoop.mapred.FileOutputCommitter
import org.apache.hadoop.mapred.FileOutputFormat
import org.apache.hadoop.mapred.HadoopWriter
import org.apache.hadoop.mapred.SparkHadoopWriter
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapred.OutputFormat
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, HadoopMapReduceUtil}
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat,
RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, SparkHadoopMapReduceUtil}
import org.apache.hadoop.security.UserGroupInformation
import spark.partial.BoundedDouble
@ -50,10 +50,9 @@ import spark.Partitioner._
* Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
* Import `spark.SparkContext._` at the top of your program to use these functions.
*/
class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
self: RDD[(K, V)])
class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
extends Logging
with HadoopMapReduceUtil
with SparkHadoopMapReduceUtil
with Serializable {
/**
@ -85,18 +84,18 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners)
if (self.partitioner == Some(partitioner)) {
self.mapPartitions(aggregator.combineValuesByKey, true)
self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true)
} else if (mapSideCombine) {
val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey, true)
val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner)
val combined = self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true)
val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner)
.setSerializer(serializerClass)
partitioned.mapPartitions(aggregator.combineCombinersByKey, true)
partitioned.mapPartitions(aggregator.combineCombinersByKey, preservesPartitioning = true)
} else {
// Don't apply map-side combiner.
// A sanity check to make sure mergeCombiners is not defined.
assert(mergeCombiners == null)
val values = new ShuffledRDD[K, V](self, partitioner).setSerializer(serializerClass)
values.mapPartitions(aggregator.combineValuesByKey, true)
val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass)
values.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true)
}
}
@ -168,7 +167,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
def reducePartition(iter: Iterator[(K, V)]): Iterator[JHashMap[K, V]] = {
val map = new JHashMap[K, V]
for ((k, v) <- iter) {
iter.foreach { case (k, v) =>
val old = map.get(k)
map.put(k, if (old == null) v else func(old, v))
}
@ -176,11 +175,11 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
def mergeMaps(m1: JHashMap[K, V], m2: JHashMap[K, V]): JHashMap[K, V] = {
for ((k, v) <- m2) {
m2.foreach { case (k, v) =>
val old = m1.get(k)
m1.put(k, if (old == null) v else func(old, v))
}
return m1
m1
}
self.mapPartitions(reducePartition).reduce(mergeMaps)
@ -240,7 +239,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) {
throw new SparkException("Default partitioner cannot partition array keys.")
}
new ShuffledRDD[K, V](self, partitioner)
new ShuffledRDD[K, V, (K, V)](self, partitioner)
}
/**
@ -249,8 +248,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
* (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD.
*/
def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = {
this.cogroup(other, partitioner).flatMapValues {
case (vs, ws) =>
this.cogroup(other, partitioner).flatMapValues { case (vs, ws) =>
for (v <- vs.iterator; w <- ws.iterator) yield (v, w)
}
}
@ -261,9 +259,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
* pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to
* partition the output RDD.
*/
def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = {
this.cogroup(other, partitioner).flatMapValues {
case (vs, ws) =>
def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner):
RDD[(K, (V, Option[W]))] = {
this.cogroup(other, partitioner).flatMapValues { case (vs, ws) =>
if (ws.isEmpty) {
vs.iterator.map(v => (v, None))
} else {
@ -280,8 +279,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
*/
def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner)
: RDD[(K, (Option[V], W))] = {
this.cogroup(other, partitioner).flatMapValues {
case (vs, ws) =>
this.cogroup(other, partitioner).flatMapValues { case (vs, ws) =>
if (vs.isEmpty) {
ws.iterator.map(w => (None, w))
} else {
@ -378,7 +376,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
/**
* Return the key-value pairs in this RDD to the master as a Map.
*/
def collectAsMap(): Map[K, V] = HashMap(self.collect(): _*)
def collectAsMap(): Map[K, V] = {
val data = self.toArray()
val map = new mutable.HashMap[K, V]
map.sizeHint(data.length)
data.foreach { case (k, v) => map.put(k, v) }
map
}
/**
* Pass each value in the key-value pair RDD through a map function without changing the keys;
@ -406,12 +410,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) {
throw new SparkException("Default partitioner cannot partition array keys.")
}
val cg = new CoGroupedRDD[K](
Seq(self.asInstanceOf[RDD[(K, _)]], other.asInstanceOf[RDD[(K, _)]]),
partitioner)
val cg = new CoGroupedRDD[K](Seq(self, other), partitioner)
val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
prfs.mapValues {
case Seq(vs, ws) =>
prfs.mapValues { case Seq(vs, ws) =>
(vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
}
}
@ -425,14 +426,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) {
throw new SparkException("Default partitioner cannot partition array keys.")
}
val cg = new CoGroupedRDD[K](
Seq(self.asInstanceOf[RDD[(K, _)]],
other1.asInstanceOf[RDD[(K, _)]],
other2.asInstanceOf[RDD[(K, _)]]),
partitioner)
val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner)
val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
prfs.mapValues {
case Seq(vs, w1s, w2s) =>
prfs.mapValues { case Seq(vs, w1s, w2s) =>
(vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]])
}
}
@ -635,7 +631,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString)
}
conf.setOutputCommitter(classOf[FileOutputCommitter])
FileOutputFormat.setOutputPath(conf, HadoopWriter.createPathFromString(path, conf))
FileOutputFormat.setOutputPath(conf, SparkHadoopWriter.createPathFromString(path, conf))
saveAsHadoopDataset(conf)
}
@ -661,10 +657,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")")
val writer = new HadoopWriter(conf)
val writer = new SparkHadoopWriter(conf)
writer.preSetup()
def writeToFile(context: TaskContext, iter: Iterator[(K,V)]) {
def writeToFile(context: TaskContext, iter: Iterator[(K, V)]) {
// Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it
// around by taking a mod. We expect that no task will be attempted 2 billion times.
val attemptNumber = (context.attemptId % Int.MaxValue).toInt
@ -714,54 +710,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure
}
/**
* Extra functions available on RDDs of (key, value) pairs where the key is sortable through
* an implicit conversion. Import `spark.SparkContext._` at the top of your program to use these
* functions. They will work with any key type that has a `scala.math.Ordered` implementation.
*/
class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
self: RDD[(K, V)])
extends Logging
with Serializable {
/**
* Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling
* `collect` or `save` on the resulting RDD will return or output an ordered list of records
* (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in
* order of the keys).
*/
def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[(K,V)] = {
val shuffled =
new ShuffledRDD[K, V](self, new RangePartitioner(numPartitions, self, ascending))
shuffled.mapPartitions(iter => {
val buf = iter.toArray
if (ascending) {
buf.sortWith((x, y) => x._1 < y._1).iterator
} else {
buf.sortWith((x, y) => x._1 > y._1).iterator
}
}, true)
}
}
private[spark]
class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev) {
override def getPartitions = firstParent[(K, V)].partitions
override val partitioner = firstParent[(K, V)].partitioner
override def compute(split: Partition, context: TaskContext) =
firstParent[(K, V)].iterator(split, context).map{ case (k, v) => (k, f(v)) }
}
private[spark]
class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U])
extends RDD[(K, U)](prev) {
override def getPartitions = firstParent[(K, V)].partitions
override val partitioner = firstParent[(K, V)].partitioner
override def compute(split: Partition, context: TaskContext) = {
firstParent[(K, V)].iterator(split, context).flatMap { case (k, v) => f(v).map(x => (k, x)) }
}
}
private[spark] object Manifests {
val seqSeqManifest = classManifest[Seq[Seq[_]]]

View file

@ -84,7 +84,7 @@ class HashPartitioner(partitions: Int) extends Partitioner {
*/
class RangePartitioner[K <% Ordered[K]: ClassManifest, V](
partitions: Int,
@transient rdd: RDD[(K,V)],
@transient rdd: RDD[_ <: Product2[K,V]],
private val ascending: Boolean = true)
extends Partitioner {

View file

@ -31,9 +31,8 @@ import org.apache.hadoop.mapred.TextOutputFormat
import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
import spark.api.java.JavaRDD
import spark.broadcast.Broadcast
import spark.Partitioner._
import spark.api.java.JavaRDD
import spark.partial.BoundedDouble
import spark.partial.CountEvaluator
import spark.partial.GroupedCountEvaluator
@ -224,8 +223,8 @@ abstract class RDD[T: ClassManifest](
}
/**
* Get the preferred location of a split, taking into account whether the
* RDD is checkpointed or not.
* Get the preferred locations of a partition (as hostnames), taking into account whether the
* RDD is checkpointed.
*/
final def preferredLocations(split: Partition): Seq[String] = {
checkpointRDD.map(_.getPreferredLocations(split)).getOrElse {
@ -290,7 +289,10 @@ abstract class RDD[T: ClassManifest](
def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = {
if (shuffle) {
// include a shuffle step so that our upstream tasks are still distributed
new CoalescedRDD(new ShuffledRDD(map(x => (x, null)), new HashPartitioner(numPartitions)), numPartitions).keys
new CoalescedRDD(
new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)),
new HashPartitioner(numPartitions)),
numPartitions).keys
} else {
new CoalescedRDD(this, numPartitions)
}
@ -305,8 +307,8 @@ abstract class RDD[T: ClassManifest](
def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = {
var fraction = 0.0
var total = 0
var multiplier = 3.0
var initialCount = this.count()
val multiplier = 3.0
val initialCount = this.count()
var maxSelected = 0
if (num < 0) {

View file

@ -22,12 +22,13 @@ import spark.serializer.Serializer
private[spark] abstract class ShuffleFetcher {
/**
* Fetch the shuffle outputs for a given ShuffleDependency.
* @return An iterator over the elements of the fetched shuffle outputs.
*/
def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics,
serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[(K,V)]
def fetch[T](shuffleId: Int, reduceId: Int, metrics: TaskMetrics,
serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[T]
/** Stop the fetcher */
def stop() {}

View file

@ -20,19 +20,14 @@ package spark
import java.io._
import java.net.URI
import java.util.Properties
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicInteger
import scala.collection.JavaConversions._
import scala.collection.Map
import scala.collection.generic.Growable
import scala.collection.mutable.HashMap
import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
import scala.util.DynamicVariable
import scala.collection.mutable.{ConcurrentMap, HashMap}
import akka.actor.Actor._
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
@ -54,23 +49,25 @@ import org.apache.hadoop.mapred.TextInputFormat
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import org.apache.hadoop.mapreduce.{Job => NewHadoopJob}
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
import org.apache.hadoop.security.UserGroupInformation
import org.apache.mesos.MesosNativeLibrary
import spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
import spark.deploy.LocalSparkCluster
import spark.partial.{ApproximateEvaluator, PartialResult}
import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD}
import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener,
SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob}
import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD,
OrderedRDDFunctions}
import spark.scheduler._
import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend,
ClusterScheduler, Schedulable, SchedulingMode}
import spark.scheduler.local.LocalScheduler
import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource}
import spark.ui.SparkUI
import spark.util.{MetadataCleaner, TimeStampedHashMap}
import ui.{SparkUI}
import spark.metrics._
import scala.Some
import spark.scheduler.StageInfo
import spark.storage.RDDInfo
import spark.storage.StorageStatus
/**
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
@ -241,7 +238,8 @@ class SparkContext(
/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
val hadoopConfiguration = {
val conf = SparkHadoopUtil.newConfiguration()
val env = SparkEnv.get
val conf = env.hadoop.newConfiguration()
// Explicitly check for S3 environment variables
if (System.getenv("AWS_ACCESS_KEY_ID") != null && System.getenv("AWS_SECRET_ACCESS_KEY") != null) {
conf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
@ -619,6 +617,16 @@ class SparkContext(
addedFiles.clear()
}
/**
* Gets the locality information associated with the partition in a particular rdd
* @param rdd of interest
* @param partition to be looked up for locality
* @return list of preferred locations for the partition
*/
private [spark] def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = {
dagScheduler.getPreferredLocs(rdd, partition)
}
/**
* Adds a JAR dependency for all tasks to be executed on this SparkContext in the future.
* The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
@ -629,10 +637,11 @@ class SparkContext(
logWarning("null specified as parameter to addJar",
new SparkException("null specified as parameter to addJar"))
} else {
val env = SparkEnv.get
val uri = new URI(path)
val key = uri.getScheme match {
case null | "file" =>
if (SparkHadoopUtil.isYarnMode()) {
if (env.hadoop.isYarnMode()) {
logWarning("local jar specified as parameter to addJar under Yarn mode")
return
}
@ -815,8 +824,9 @@ class SparkContext(
* prevent accidental overriding of checkpoint files in the existing directory.
*/
def setCheckpointDir(dir: String, useExisting: Boolean = false) {
val env = SparkEnv.get
val path = new Path(dir)
val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration())
val fs = path.getFileSystem(env.hadoop.newConfiguration())
if (!useExisting) {
if (fs.exists(path)) {
throw new Exception("Checkpoint directory '" + path + "' already exists.")
@ -833,11 +843,11 @@ class SparkContext(
/** Default min number of partitions for Hadoop RDDs when not given by user */
def defaultMinSplits: Int = math.min(defaultParallelism, 2)
private var nextShuffleId = new AtomicInteger(0)
private val nextShuffleId = new AtomicInteger(0)
private[spark] def newShuffleId(): Int = nextShuffleId.getAndIncrement()
private var nextRddId = new AtomicInteger(0)
private val nextRddId = new AtomicInteger(0)
/** Register a new RDD, returning its RDD ID */
private[spark] def newRddId(): Int = nextRddId.getAndIncrement()
@ -886,7 +896,7 @@ object SparkContext {
implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
rdd: RDD[(K, V)]) =
new OrderedRDDFunctions(rdd)
new OrderedRDDFunctions[K, V, (K, V)](rdd)
implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd)

View file

@ -25,6 +25,7 @@ import akka.remote.RemoteActorRefProvider
import spark.broadcast.BroadcastManager
import spark.metrics.MetricsSystem
import spark.deploy.SparkHadoopUtil
import spark.storage.BlockManager
import spark.storage.BlockManagerMaster
import spark.network.ConnectionManager
@ -54,14 +55,23 @@ class SparkEnv (
val connectionManager: ConnectionManager,
val httpFileServer: HttpFileServer,
val sparkFilesDir: String,
val metricsSystem: MetricsSystem,
// To be set only as part of initialization of SparkContext.
// (executorId, defaultHostPort) => executorHostPort
// If executorId is NOT found, return defaultHostPort
var executorIdToHostPort: Option[(String, String) => String]) {
val metricsSystem: MetricsSystem) {
private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]()
val hadoop = {
val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
if(yarnMode) {
try {
Class.forName("spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil]
} catch {
case th: Throwable => throw new SparkException("Unable to load YARN support", th)
}
} else {
new SparkHadoopUtil
}
}
def stop() {
pythonWorkers.foreach { case(key, worker) => worker.stop() }
httpFileServer.stop()
@ -83,16 +93,6 @@ class SparkEnv (
pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create()
}
}
def resolveExecutorIdToHostPort(executorId: String, defaultHostPort: String): String = {
val env = SparkEnv.get
if (env.executorIdToHostPort.isEmpty) {
// default to using host, not host port. Relevant to non cluster modes.
return defaultHostPort
}
env.executorIdToHostPort.get(executorId, defaultHostPort)
}
}
object SparkEnv extends Logging {
@ -236,7 +236,6 @@ object SparkEnv extends Logging {
connectionManager,
httpFileServer,
sparkFilesDir,
metricsSystem,
None)
metricsSystem)
}
}

View file

@ -36,7 +36,7 @@ import spark.SerializableWritable
* Saves the RDD using a JobConf, which should contain an output key class, an output value class,
* a filename to write to, etc, exactly like in a Hadoop MapReduce job.
*/
class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRedUtil with Serializable {
class SparkHadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoopMapRedUtil with Serializable {
private val now = new Date()
private val conf = new SerializableWritable(jobConf)
@ -165,7 +165,7 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe
splitID = splitid
attemptID = attemptid
jID = new SerializableWritable[JobID](HadoopWriter.createJobID(now, jobid))
jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobid))
taID = new SerializableWritable[TaskAttemptID](
new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID))
}
@ -179,7 +179,7 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRe
}
}
object HadoopWriter {
object SparkHadoopWriter {
def createJobID(time: Date, id: Int): JobID = {
val formatter = new SimpleDateFormat("yyyyMMddHHmm")
val jobtrackerID = formatter.format(new Date())

View file

@ -266,8 +266,9 @@ private object Utils extends Logging {
}
case _ =>
// Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others
val env = SparkEnv.get
val uri = new URI(url)
val conf = SparkHadoopUtil.newConfiguration()
val conf = env.hadoop.newConfiguration()
val fs = FileSystem.get(uri, conf)
val in = fs.open(new Path(uri))
val out = new FileOutputStream(tempFile)
@ -393,48 +394,17 @@ private object Utils extends Logging {
retval
}
/*
// Used by DEBUG code : remove when all testing done
private val ipPattern = Pattern.compile("^[0-9]+(\\.[0-9]+)*$")
def checkHost(host: String, message: String = "") {
// Currently catches only ipv4 pattern, this is just a debugging tool - not rigourous !
// if (host.matches("^[0-9]+(\\.[0-9]+)*$")) {
if (ipPattern.matcher(host).matches()) {
Utils.logErrorWithStack("Unexpected to have host " + host + " which matches IP pattern. Message " + message)
}
if (Utils.parseHostPort(host)._2 != 0){
Utils.logErrorWithStack("Unexpected to have host " + host + " which has port in it. Message " + message)
}
assert(host.indexOf(':') == -1, message)
}
// Used by DEBUG code : remove when all testing done
def checkHostPort(hostPort: String, message: String = "") {
val (host, port) = Utils.parseHostPort(hostPort)
checkHost(host)
if (port <= 0){
Utils.logErrorWithStack("Unexpected to have port " + port + " which is not valid in " + hostPort + ". Message " + message)
}
assert(hostPort.indexOf(':') != -1, message)
}
// Used by DEBUG code : remove when all testing done
def logErrorWithStack(msg: String) {
try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } }
// temp code for debug
System.exit(-1)
}
*/
// Once testing is complete in various modes, replace with this ?
def checkHost(host: String, message: String = "") {}
def checkHostPort(hostPort: String, message: String = "") {}
// Used by DEBUG code : remove when all testing done
def logErrorWithStack(msg: String) {
try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } }
}
def getUserNameFromEnvironment(): String = {
SparkHadoopUtil.getUserNameFromEnvironment
}
// Typically, this will be of order of number of nodes in cluster

View file

@ -30,17 +30,18 @@ import org.apache.hadoop.mapred.OutputFormat
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
import org.apache.hadoop.conf.Configuration
import spark.api.java.function.{Function2 => JFunction2}
import spark.api.java.function.{Function => JFunction}
import spark.partial.BoundedDouble
import spark.partial.PartialResult
import spark.OrderedRDDFunctions
import spark.storage.StorageLevel
import spark.HashPartitioner
import spark.Partitioner
import spark.Partitioner._
import spark.RDD
import spark.SparkContext.rddToPairRDDFunctions
import spark.api.java.function.{Function2 => JFunction2}
import spark.api.java.function.{Function => JFunction}
import spark.partial.BoundedDouble
import spark.partial.PartialResult
import spark.rdd.OrderedRDDFunctions
import spark.storage.StorageLevel
class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManifest[K],
implicit val vManifest: ClassManifest[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] {
@ -615,7 +616,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
override def compare(b: K) = comp.compare(a, b)
}
implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x)
fromRDD(new OrderedRDDFunctions(rdd).sortByKey(ascending))
fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending))
}
/**

View file

@ -80,6 +80,7 @@ private[deploy] object DeployMessages {
case class RegisteredApplication(appId: String) extends DeployMessage
// TODO(matei): replace hostPort with host
case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) {
Utils.checkHostPort(hostPort, "Required hostport")
}

View file

@ -17,7 +17,7 @@
package spark.deploy
import scala.util.parsing.json.{JSONArray, JSONObject, JSONType}
import net.liftweb.json.JsonDSL._
import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
import spark.deploy.master.{ApplicationInfo, WorkerInfo}
@ -25,63 +25,62 @@ import spark.deploy.worker.ExecutorRunner
private[spark] object JsonProtocol {
def writeWorkerInfo(obj: WorkerInfo) = {
("id" -> obj.id) ~
("host" -> obj.host) ~
("port" -> obj.port) ~
("webuiaddress" -> obj.webUiAddress) ~
("cores" -> obj.cores) ~
("coresused" -> obj.coresUsed) ~
("memory" -> obj.memory) ~
("memoryused" -> obj.memoryUsed) ~
("state" -> obj.state.toString)
}
def writeWorkerInfo(obj: WorkerInfo): JSONType = JSONObject(Map(
"id" -> obj.id,
"host" -> obj.host,
"port" -> obj.port,
"webuiaddress" -> obj.webUiAddress,
"cores" -> obj.cores,
"coresused" -> obj.coresUsed,
"memory" -> obj.memory,
"memoryused" -> obj.memoryUsed,
"state" -> obj.state.toString
))
def writeApplicationInfo(obj: ApplicationInfo) = {
("starttime" -> obj.startTime) ~
("id" -> obj.id) ~
("name" -> obj.desc.name) ~
("cores" -> obj.desc.maxCores) ~
("user" -> obj.desc.user) ~
("memoryperslave" -> obj.desc.memoryPerSlave) ~
("submitdate" -> obj.submitDate.toString)
}
def writeApplicationInfo(obj: ApplicationInfo): JSONType = JSONObject(Map(
"starttime" -> obj.startTime,
"id" -> obj.id,
"name" -> obj.desc.name,
"cores" -> obj.desc.maxCores,
"user" -> obj.desc.user,
"memoryperslave" -> obj.desc.memoryPerSlave,
"submitdate" -> obj.submitDate.toString
))
def writeApplicationDescription(obj: ApplicationDescription) = {
("name" -> obj.name) ~
("cores" -> obj.maxCores) ~
("memoryperslave" -> obj.memoryPerSlave) ~
("user" -> obj.user)
}
def writeApplicationDescription(obj: ApplicationDescription): JSONType = JSONObject(Map(
"name" -> obj.name,
"cores" -> obj.maxCores,
"memoryperslave" -> obj.memoryPerSlave,
"user" -> obj.user
))
def writeExecutorRunner(obj: ExecutorRunner) = {
("id" -> obj.execId) ~
("memory" -> obj.memory) ~
("appid" -> obj.appId) ~
("appdesc" -> writeApplicationDescription(obj.appDesc))
}
def writeExecutorRunner(obj: ExecutorRunner): JSONType = JSONObject(Map(
"id" -> obj.execId,
"memory" -> obj.memory,
"appid" -> obj.appId,
"appdesc" -> writeApplicationDescription(obj.appDesc)
))
def writeMasterState(obj: MasterStateResponse) = {
("url" -> ("spark://" + obj.uri)) ~
("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~
("cores" -> obj.workers.map(_.cores).sum) ~
("coresused" -> obj.workers.map(_.coresUsed).sum) ~
("memory" -> obj.workers.map(_.memory).sum) ~
("memoryused" -> obj.workers.map(_.memoryUsed).sum) ~
("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~
("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo))
}
def writeMasterState(obj: MasterStateResponse): JSONType = JSONObject(Map(
"url" -> ("spark://" + obj.uri),
"workers" -> obj.workers.toList.map(writeWorkerInfo),
"cores" -> obj.workers.map(_.cores).sum,
"coresused" -> obj.workers.map(_.coresUsed).sum,
"memory" -> obj.workers.map(_.memory).sum,
"memoryused" -> obj.workers.map(_.memoryUsed).sum,
"activeapps" -> JSONArray(obj.activeApps.toList.map(writeApplicationInfo)),
"completedapps" -> JSONArray(obj.completedApps.toList.map(writeApplicationInfo))
))
def writeWorkerState(obj: WorkerStateResponse): JSONType = JSONObject(Map(
"id" -> obj.workerId,
"masterurl" -> obj.masterUrl,
"masterwebuiurl" -> obj.masterWebUiUrl,
"cores" -> obj.cores,
"coresused" -> obj.coresUsed,
"memory" -> obj.memory,
"memoryused" -> obj.memoryUsed,
"executors" -> JSONArray(obj.executors.toList.map(writeExecutorRunner)),
"finishedexecutors" -> JSONArray(obj.finishedExecutors.toList.map(writeExecutorRunner))
))
def writeWorkerState(obj: WorkerStateResponse) = {
("id" -> obj.workerId) ~
("masterurl" -> obj.masterUrl) ~
("masterwebuiurl" -> obj.masterWebUiUrl) ~
("cores" -> obj.cores) ~
("coresused" -> obj.coresUsed) ~
("memory" -> obj.memory) ~
("memoryused" -> obj.memoryUsed) ~
("executors" -> obj.executors.toList.map(writeExecutorRunner)) ~
("finishedexecutors" -> obj.finishedExecutors.toList.map(writeExecutorRunner))
}
}

View file

@ -23,18 +23,7 @@ import org.apache.hadoop.mapred.JobConf
/**
* Contains util methods to interact with Hadoop from spark.
*/
object SparkHadoopUtil {
def getUserNameFromEnvironment(): String = {
// defaulting to -D ...
System.getProperty("user.name")
}
def runAsUser(func: (Product) => Unit, args: Product) {
// Add support, if exists - for now, simply run func !
func(args)
}
class SparkHadoopUtil {
// Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
def newConfiguration(): Configuration = new Configuration()

View file

@ -17,7 +17,6 @@
package spark.deploy.master.ui
import scala.util.parsing.json.JSONType
import scala.xml.Node
import akka.dispatch.Await
@ -26,6 +25,8 @@ import akka.util.duration._
import javax.servlet.http.HttpServletRequest
import net.liftweb.json.JsonAST.JValue
import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
import spark.deploy.JsonProtocol
import spark.deploy.master.ExecutorInfo
@ -36,7 +37,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {
implicit val timeout = parent.timeout
/** Executor details for a particular application */
def renderJson(request: HttpServletRequest): JSONType = {
def renderJson(request: HttpServletRequest): JValue = {
val appId = request.getParameter("appId")
val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
val state = Await.result(stateFuture, 30 seconds)

View file

@ -19,13 +19,14 @@ package spark.deploy.master.ui
import javax.servlet.http.HttpServletRequest
import scala.util.parsing.json.JSONType
import scala.xml.Node
import akka.dispatch.Await
import akka.pattern.ask
import akka.util.duration._
import net.liftweb.json.JsonAST.JValue
import spark.Utils
import spark.deploy.DeployWebUI
import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
@ -37,7 +38,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
val master = parent.masterActorRef
implicit val timeout = parent.timeout
def renderJson(request: HttpServletRequest): JSONType = {
def renderJson(request: HttpServletRequest): JValue = {
val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
val state = Await.result(stateFuture, 30 seconds)
JsonProtocol.writeMasterState(state)

View file

@ -40,13 +40,11 @@ private[spark] class ExecutorRunner(
val memory: Int,
val worker: ActorRef,
val workerId: String,
val hostPort: String,
val host: String,
val sparkHome: File,
val workDir: File)
extends Logging {
Utils.checkHostPort(hostPort, "Expected hostport")
val fullId = appId + "/" + execId
var workerThread: Thread = null
var process: Process = null
@ -92,7 +90,7 @@ private[spark] class ExecutorRunner(
/** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */
def substituteVariables(argument: String): String = argument match {
case "{{EXECUTOR_ID}}" => execId.toString
case "{{HOSTNAME}}" => Utils.parseHostPort(hostPort)._1
case "{{HOSTNAME}}" => host
case "{{CORES}}" => cores.toString
case other => other
}

View file

@ -132,7 +132,7 @@ private[spark] class Worker(
case LaunchExecutor(appId, execId, appDesc, cores_, memory_, execSparkHome_) =>
logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
val manager = new ExecutorRunner(
appId, execId, appDesc, cores_, memory_, self, workerId, host + ":" + port, new File(execSparkHome_), workDir)
appId, execId, appDesc, cores_, memory_, self, workerId, host, new File(execSparkHome_), workDir)
executors(appId + "/" + execId) = manager
manager.start()
coresUsed += cores_

View file

@ -19,13 +19,14 @@ package spark.deploy.worker.ui
import javax.servlet.http.HttpServletRequest
import scala.util.parsing.json.JSONType
import scala.xml.Node
import akka.dispatch.Await
import akka.pattern.ask
import akka.util.duration._
import net.liftweb.json.JsonAST.JValue
import spark.Utils
import spark.deploy.JsonProtocol
import spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse}
@ -38,7 +39,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
val worker = parent.worker
val timeout = parent.timeout
def renderJson(request: HttpServletRequest): JSONType = {
def renderJson(request: HttpServletRequest): JValue = {
val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse]
val workerState = Await.result(stateFuture, 30 seconds)
JsonProtocol.writeWorkerState(workerState)

View file

@ -32,8 +32,12 @@ import spark._
/**
* The Mesos executor for Spark.
*/
private[spark] class Executor(executorId: String, slaveHostname: String, properties: Seq[(String, String)]) extends Logging {
private[spark] class Executor(
executorId: String,
slaveHostname: String,
properties: Seq[(String, String)])
extends Logging
{
// Application dependencies (added through SparkContext) that we've fetched so far on this node.
// Each map holds the master's timestamp for the version of that file or JAR we got.
private val currentFiles: HashMap[String, Long] = new HashMap[String, Long]()
@ -125,8 +129,8 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert
updateDependencies(taskFiles, taskJars)
val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader)
attemptedTask = Some(task)
logInfo("Its generation is " + task.generation)
env.mapOutputTracker.updateGeneration(task.generation)
logInfo("Its epoch is " + task.epoch)
env.mapOutputTracker.updateEpoch(task.epoch)
taskStart = System.currentTimeMillis()
val value = task.run(taskId.toInt)
val taskFinish = System.currentTimeMillis()

View file

@ -22,9 +22,8 @@ import java.nio.ByteBuffer
import akka.actor.{ActorRef, Actor, Props, Terminated}
import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
import spark.{Logging, Utils}
import spark.{Logging, Utils, SparkEnv}
import spark.TaskState.TaskState
import spark.deploy.SparkHadoopUtil
import spark.scheduler.cluster.StandaloneClusterMessages._
import spark.util.AkkaUtils
@ -82,19 +81,6 @@ private[spark] class StandaloneExecutorBackend(
private[spark] object StandaloneExecutorBackend {
def run(driverUrl: String, executorId: String, hostname: String, cores: Int) {
SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores))
}
// This will be run 'as' the user
def run0(args: Product) {
assert(4 == args.productArity)
runImpl(args.productElement(0).asInstanceOf[String],
args.productElement(1).asInstanceOf[String],
args.productElement(2).asInstanceOf[String],
args.productElement(3).asInstanceOf[Int])
}
private def runImpl(driverUrl: String, executorId: String, hostname: String, cores: Int) {
// Debug code
Utils.checkHost(hostname)

View file

@ -28,13 +28,12 @@ private[spark]
class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String])
extends RDD[T](sc, Nil) {
@transient lazy val locations_ = BlockManager.blockIdsToExecutorLocations(blockIds, SparkEnv.get)
@transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get)
override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => {
new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition]
}).toArray
override def compute(split: Partition, context: TaskContext): Iterator[T] = {
val blockManager = SparkEnv.get.blockManager
val blockId = split.asInstanceOf[BlockRDDPartition].blockId
@ -45,8 +44,8 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St
}
}
override def getPreferredLocations(split: Partition): Seq[String] =
override def getPreferredLocations(split: Partition): Seq[String] = {
locations_(split.asInstanceOf[BlockRDDPartition].blockId)
}
}

View file

@ -64,7 +64,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest](
override def getPreferredLocations(split: Partition): Seq[String] = {
val currSplit = split.asInstanceOf[CartesianPartition]
rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)
(rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)).distinct
}
override def compute(split: Partition, context: TaskContext) = {

View file

@ -25,7 +25,6 @@ import org.apache.hadoop.util.ReflectionUtils
import org.apache.hadoop.fs.Path
import java.io.{File, IOException, EOFException}
import java.text.NumberFormat
import spark.deploy.SparkHadoopUtil
private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {}
@ -82,8 +81,9 @@ private[spark] object CheckpointRDD extends Logging {
}
def writeToFile[T](path: String, blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) {
val env = SparkEnv.get
val outputDir = new Path(path)
val fs = outputDir.getFileSystem(SparkHadoopUtil.newConfiguration())
val fs = outputDir.getFileSystem(env.hadoop.newConfiguration())
val finalOutputName = splitIdToFile(ctx.splitId)
val finalOutputPath = new Path(outputDir, finalOutputName)
@ -101,7 +101,7 @@ private[spark] object CheckpointRDD extends Logging {
// This is mainly for testing purpose
fs.create(tempOutputPath, false, bufferSize, fs.getDefaultReplication, blockSize)
}
val serializer = SparkEnv.get.serializer.newInstance()
val serializer = env.serializer.newInstance()
val serializeStream = serializer.serializeStream(fileOutputStream)
serializeStream.writeAll(iterator)
serializeStream.close()
@ -121,10 +121,11 @@ private[spark] object CheckpointRDD extends Logging {
}
def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = {
val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration())
val env = SparkEnv.get
val fs = path.getFileSystem(env.hadoop.newConfiguration())
val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
val fileInputStream = fs.open(path, bufferSize)
val serializer = SparkEnv.get.serializer.newInstance()
val serializer = env.serializer.newInstance()
val deserializeStream = serializer.deserializeStream(fileInputStream)
// Register an on-task-completion callback to close the input stream.
@ -140,10 +141,11 @@ private[spark] object CheckpointRDD extends Logging {
import spark._
val Array(cluster, hdfsPath) = args
val env = SparkEnv.get
val sc = new SparkContext(cluster, "CheckpointRDD Test")
val rdd = sc.makeRDD(1 to 10, 10).flatMap(x => 1 to 10000)
val path = new Path(hdfsPath, "temp")
val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration())
val fs = path.getFileSystem(env.hadoop.newConfiguration())
sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _)
val cpRDD = new CheckpointRDD[Int](sc, path.toString)
assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same")

View file

@ -60,7 +60,7 @@ class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep])
* @param rdds parent RDDs.
* @param part partitioner used to partition the shuffle output.
*/
class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner)
extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) {
private var serializerClass: String = null
@ -71,13 +71,13 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
}
override def getDependencies: Seq[Dependency[_]] = {
rdds.map { rdd: RDD[(K, _)] =>
rdds.map { rdd: RDD[_ <: Product2[K, _]] =>
if (rdd.partitioner == Some(part)) {
logInfo("Adding one-to-one dependency with " + rdd)
logDebug("Adding one-to-one dependency with " + rdd)
new OneToOneDependency(rdd)
} else {
logInfo("Adding shuffle dependency with " + rdd)
new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part, serializerClass)
logDebug("Adding shuffle dependency with " + rdd)
new ShuffleDependency[Any, Any](rdd, part, serializerClass)
}
}
}
@ -122,15 +122,15 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
for ((dep, depNum) <- split.deps.zipWithIndex) dep match {
case NarrowCoGroupSplitDep(rdd, _, itsSplit) => {
// Read them from the parent
for ((k, v) <- rdd.iterator(itsSplit, context)) {
getSeq(k.asInstanceOf[K])(depNum) += v
rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { kv =>
getSeq(kv._1)(depNum) += kv._2
}
}
case ShuffleCoGroupSplitDep(shuffleId) => {
// Read map outputs of shuffle
val fetcher = SparkEnv.get.shuffleFetcher
fetcher.fetch[K, Any](shuffleId, split.index, context.taskMetrics, ser).foreach {
case (key, value) => getSeq(key)(depNum) += value
fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context.taskMetrics, ser).foreach {
kv => getSeq(kv._1)(depNum) += kv._2
}
}
}

View file

@ -17,53 +17,76 @@
package spark.rdd
import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext}
import spark._
import java.io.{ObjectOutputStream, IOException}
import scala.collection.mutable
import scala.Some
import scala.collection.mutable.ArrayBuffer
private[spark] case class CoalescedRDDPartition(
/**
* Class that captures a coalesced RDD by essentially keeping track of parent partitions
* @param index of this coalesced partition
* @param rdd which it belongs to
* @param parentsIndices list of indices in the parent that have been coalesced into this partition
* @param preferredLocation the preferred location for this partition
*/
case class CoalescedRDDPartition(
index: Int,
@transient rdd: RDD[_],
parentsIndices: Array[Int]
parentsIndices: Array[Int],
@transient preferredLocation: String = ""
) extends Partition {
var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_))
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
// Update the reference to parent partition at the time of task serialization
parents = parentsIndices.map(rdd.partitions(_))
oos.defaultWriteObject()
}
/**
* Computes how many of the parents partitions have getPreferredLocation
* as one of their preferredLocations
* @return locality of this coalesced partition between 0 and 1
*/
def localFraction: Double = {
val loc = parents.count(p =>
rdd.context.getPreferredLocs(rdd, p.index).map(tl => tl.host).contains(preferredLocation))
if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble)
}
}
/**
* Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of
* this RDD computes one or more of the parent ones. Will produce exactly `maxPartitions` if the
* parent had more than this many partitions, or fewer if the parent had fewer.
*
* This transformation is useful when an RDD with many partitions gets filtered into a smaller one,
* or to avoid having a large number of small tasks when processing a directory with many files.
* Represents a coalesced RDD that has fewer partitions than its parent RDD
* This class uses the PartitionCoalescer class to find a good partitioning of the parent RDD
* so that each new partition has roughly the same number of parent partitions and that
* the preferred location of each new partition overlaps with as many preferred locations of its
* parent partitions
* @param prev RDD to be coalesced
* @param maxPartitions number of desired partitions in the coalesced RDD
* @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance
*/
class CoalescedRDD[T: ClassManifest](
@transient var prev: RDD[T],
maxPartitions: Int)
maxPartitions: Int,
balanceSlack: Double = 0.10)
extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies
override def getPartitions: Array[Partition] = {
val prevSplits = prev.partitions
if (prevSplits.length < maxPartitions) {
prevSplits.map(_.index).map{idx => new CoalescedRDDPartition(idx, prev, Array(idx)) }
} else {
(0 until maxPartitions).map { i =>
val rangeStart = ((i.toLong * prevSplits.length) / maxPartitions).toInt
val rangeEnd = (((i.toLong + 1) * prevSplits.length) / maxPartitions).toInt
new CoalescedRDDPartition(i, prev, (rangeStart until rangeEnd).toArray)
}.toArray
val pc = new PartitionCoalescer(maxPartitions, prev, balanceSlack)
pc.run().zipWithIndex.map {
case (pg, i) =>
val ids = pg.arr.map(_.index).toArray
new CoalescedRDDPartition(i, prev, ids, pg.prefLoc)
}
}
override def compute(split: Partition, context: TaskContext): Iterator[T] = {
split.asInstanceOf[CoalescedRDDPartition].parents.iterator.flatMap { parentSplit =>
firstParent[T].iterator(parentSplit, context)
override def compute(partition: Partition, context: TaskContext): Iterator[T] = {
partition.asInstanceOf[CoalescedRDDPartition].parents.iterator.flatMap { parentPartition =>
firstParent[T].iterator(parentPartition, context)
}
}
@ -78,4 +101,242 @@ class CoalescedRDD[T: ClassManifest](
super.clearDependencies()
prev = null
}
/**
* Returns the preferred machine for the partition. If split is of type CoalescedRDDPartition,
* then the preferred machine will be one which most parent splits prefer too.
* @param partition
* @return the machine most preferred by split
*/
override def getPreferredLocations(partition: Partition): Seq[String] = {
List(partition.asInstanceOf[CoalescedRDDPartition].preferredLocation)
}
}
/**
* Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of
* this RDD computes one or more of the parent ones. It will produce exactly `maxPartitions` if the
* parent had more than maxPartitions, or fewer if the parent had fewer.
*
* This transformation is useful when an RDD with many partitions gets filtered into a smaller one,
* or to avoid having a large number of small tasks when processing a directory with many files.
*
* If there is no locality information (no preferredLocations) in the parent, then the coalescing
* is very simple: chunk parents that are close in the Array in chunks.
* If there is locality information, it proceeds to pack them with the following four goals:
*
* (1) Balance the groups so they roughly have the same number of parent partitions
* (2) Achieve locality per partition, i.e. find one machine which most parent partitions prefer
* (3) Be efficient, i.e. O(n) algorithm for n parent partitions (problem is likely NP-hard)
* (4) Balance preferred machines, i.e. avoid as much as possible picking the same preferred machine
*
* Furthermore, it is assumed that the parent RDD may have many partitions, e.g. 100 000.
* We assume the final number of desired partitions is small, e.g. less than 1000.
*
* The algorithm tries to assign unique preferred machines to each partition. If the number of
* desired partitions is greater than the number of preferred machines (can happen), it needs to
* start picking duplicate preferred machines. This is determined using coupon collector estimation
* (2n log(n)). The load balancing is done using power-of-two randomized bins-balls with one twist:
* it tries to also achieve locality. This is done by allowing a slack (balanceSlack) between two
* bins. If two bins are within the slack in terms of balance, the algorithm will assign partitions
* according to locality. (contact alig for questions)
*
*/
private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) {
def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size
def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean =
if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get)
val rnd = new scala.util.Random(7919) // keep this class deterministic
// each element of groupArr represents one coalesced partition
val groupArr = ArrayBuffer[PartitionGroup]()
// hash used to check whether some machine is already in groupArr
val groupHash = mutable.Map[String, ArrayBuffer[PartitionGroup]]()
// hash used for the first maxPartitions (to avoid duplicates)
val initialHash = mutable.Set[Partition]()
// determines the tradeoff between load-balancing the partitions sizes and their locality
// e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality
val slack = (balanceSlack * prev.partitions.size).toInt
var noLocality = true // if true if no preferredLocations exists for parent RDD
// gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones)
def currPrefLocs(part: Partition): Seq[String] = {
prev.context.getPreferredLocs(prev, part.index).map(tl => tl.host)
}
// this class just keeps iterating and rotating infinitely over the partitions of the RDD
// next() returns the next preferred machine that a partition is replicated on
// the rotator first goes through the first replica copy of each partition, then second, third
// the iterators return type is a tuple: (replicaString, partition)
class LocationIterator(prev: RDD[_]) extends Iterator[(String, Partition)] {
var it: Iterator[(String, Partition)] = resetIterator()
override val isEmpty = !it.hasNext
// initializes/resets to start iterating from the beginning
def resetIterator() = {
val iterators = (0 to 2).map( x =>
prev.partitions.iterator.flatMap(p => {
if (currPrefLocs(p).size > x) Some((currPrefLocs(p)(x), p)) else None
} )
)
iterators.reduceLeft((x, y) => x ++ y)
}
// hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD
def hasNext(): Boolean = { !isEmpty }
// return the next preferredLocation of some partition of the RDD
def next(): (String, Partition) = {
if (it.hasNext)
it.next()
else {
it = resetIterator() // ran out of preferred locations, reset and rotate to the beginning
it.next()
}
}
}
/**
* Sorts and gets the least element of the list associated with key in groupHash
* The returned PartitionGroup is the least loaded of all groups that represent the machine "key"
* @param key string representing a partitioned group on preferred machine key
* @return Option of PartitionGroup that has least elements for key
*/
def getLeastGroupHash(key: String): Option[PartitionGroup] = {
groupHash.get(key).map(_.sortWith(compare).head)
}
def addPartToPGroup(part: Partition, pgroup: PartitionGroup): Boolean = {
if (!initialHash.contains(part)) {
pgroup.arr += part // already assign this element
initialHash += part // needed to avoid assigning partitions to multiple buckets
true
} else { false }
}
/**
* Initializes targetLen partition groups and assigns a preferredLocation
* This uses coupon collector to estimate how many preferredLocations it must rotate through
* until it has seen most of the preferred locations (2 * n log(n))
* @param targetLen
*/
def setupGroups(targetLen: Int) {
val rotIt = new LocationIterator(prev)
// deal with empty case, just create targetLen partition groups with no preferred location
if (!rotIt.hasNext()) {
(1 to targetLen).foreach(x => groupArr += PartitionGroup())
return
}
noLocality = false
// number of iterations needed to be certain that we've seen most preferred locations
val expectedCoupons2 = 2 * (math.log(targetLen)*targetLen + targetLen + 0.5).toInt
var numCreated = 0
var tries = 0
// rotate through until either targetLen unique/distinct preferred locations have been created
// OR we've rotated expectedCoupons2, in which case we have likely seen all preferred locations,
// i.e. likely targetLen >> number of preferred locations (more buckets than there are machines)
while (numCreated < targetLen && tries < expectedCoupons2) {
tries += 1
val (nxt_replica, nxt_part) = rotIt.next()
if (!groupHash.contains(nxt_replica)) {
val pgroup = PartitionGroup(nxt_replica)
groupArr += pgroup
addPartToPGroup(nxt_part, pgroup)
groupHash += (nxt_replica -> (ArrayBuffer(pgroup))) // list in case we have multiple
numCreated += 1
}
}
while (numCreated < targetLen) { // if we don't have enough partition groups, create duplicates
var (nxt_replica, nxt_part) = rotIt.next()
val pgroup = PartitionGroup(nxt_replica)
groupArr += pgroup
groupHash.get(nxt_replica).get += pgroup
var tries = 0
while (!addPartToPGroup(nxt_part, pgroup) && tries < targetLen) { // ensure at least one part
nxt_part = rotIt.next()._2
tries += 1
}
numCreated += 1
}
}
/**
* Takes a parent RDD partition and decides which of the partition groups to put it in
* Takes locality into account, but also uses power of 2 choices to load balance
* It strikes a balance between the two use the balanceSlack variable
* @param p partition (ball to be thrown)
* @return partition group (bin to be put in)
*/
def pickBin(p: Partition): PartitionGroup = {
val pref = currPrefLocs(p).map(getLeastGroupHash(_)).sortWith(compare) // least loaded pref locs
val prefPart = if (pref == Nil) None else pref.head
val r1 = rnd.nextInt(groupArr.size)
val r2 = rnd.nextInt(groupArr.size)
val minPowerOfTwo = if (groupArr(r1).size < groupArr(r2).size) groupArr(r1) else groupArr(r2)
if (prefPart== None) // if no preferred locations, just use basic power of two
return minPowerOfTwo
val prefPartActual = prefPart.get
if (minPowerOfTwo.size + slack <= prefPartActual.size) // more imbalance than the slack allows
return minPowerOfTwo // prefer balance over locality
else {
return prefPartActual // prefer locality over balance
}
}
def throwBalls() {
if (noLocality) { // no preferredLocations in parent RDD, no randomization needed
if (maxPartitions > groupArr.size) { // just return prev.partitions
for ((p,i) <- prev.partitions.zipWithIndex) {
groupArr(i).arr += p
}
} else { // no locality available, then simply split partitions based on positions in array
for(i <- 0 until maxPartitions) {
val rangeStart = ((i.toLong * prev.partitions.length) / maxPartitions).toInt
val rangeEnd = (((i.toLong + 1) * prev.partitions.length) / maxPartitions).toInt
(rangeStart until rangeEnd).foreach{ j => groupArr(i).arr += prev.partitions(j) }
}
}
} else {
for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into group
pickBin(p).arr += p
}
}
}
def getPartitions: Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray
/**
* Runs the packing algorithm and returns an array of PartitionGroups that if possible are
* load balanced and grouped by locality
* @return array of partition groups
*/
def run(): Array[PartitionGroup] = {
setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins)
throwBalls() // assign partitions (balls) to each group (bins)
getPartitions
}
}
private[spark] case class PartitionGroup(prefLoc: String = "") {
var arr = mutable.ArrayBuffer[Partition]()
def size = arr.size
}

View file

@ -1,4 +1,3 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -16,15 +15,22 @@
* limitations under the License.
*/
package org.apache.hadoop.mapred
package spark.rdd
import org.apache.hadoop.mapreduce.TaskType
import spark.{TaskContext, Partition, RDD}
trait HadoopMapRedUtil {
def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
private[spark]
class FlatMappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => TraversableOnce[U])
extends RDD[(K, U)](prev) {
def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) =
new TaskAttemptID(jtIdentifier, jobId, if (isMap) TaskType.MAP else TaskType.REDUCE, taskId, attemptId)
override def getPartitions = firstParent[Product2[K, V]].partitions
override val partitioner = firstParent[Product2[K, V]].partitioner
override def compute(split: Partition, context: TaskContext) = {
firstParent[Product2[K, V]].iterator(split, context).flatMap { case Product2(k, v) =>
f(v).map(x => (k, x))
}
}
}

View file

@ -32,8 +32,7 @@ import org.apache.hadoop.mapred.RecordReader
import org.apache.hadoop.mapred.Reporter
import org.apache.hadoop.util.ReflectionUtils
import spark.deploy.SparkHadoopUtil
import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext}
import spark.util.NextIterator
import org.apache.hadoop.conf.Configurable
@ -68,7 +67,8 @@ class HadoopRDD[K, V](
private val confBroadcast = sc.broadcast(new SerializableWritable(conf))
override def getPartitions: Array[Partition] = {
SparkHadoopUtil.addCredentials(conf);
val env = SparkEnv.get
env.hadoop.addCredentials(conf)
val inputFormat = createInputFormat(conf)
if (inputFormat.isInstanceOf[Configurable]) {
inputFormat.asInstanceOf[Configurable].setConf(conf)

View file

@ -147,7 +147,7 @@ object IndexedRDD {
// Shuffle the table (if necessary)
val shuffledTbl =
if (tbl.partitioner.isEmpty) {
new ShuffledRDD[K,V](tbl, Partitioner.defaultPartitioner(tbl))
new ShuffledRDD[K, V, (K,V)](tbl, Partitioner.defaultPartitioner(tbl))
} else { tbl }
val groups = shuffledTbl.mapPartitions( iter => {
@ -180,7 +180,7 @@ object IndexedRDD {
if (tbl.partitioner == Some(partitioner)) {
tbl
} else {
new ShuffledRDD[K,V](tbl, partitioner)
new ShuffledRDD[K, V, (K,V)](tbl, partitioner)
}
// Use the index to build the new values table
@ -213,15 +213,15 @@ object IndexedRDD {
*/
def makeIndex[K: ClassManifest](keys: RDD[K],
partitioner: Option[Partitioner] = None): RDDIndex[K] = {
// @todo: I don't need the boolean its only there to be the second type since I want to shuffle a single RDD
// Ugly hack :-(. In order to partition the keys they must have values.
val tbl = keys.mapPartitions(_.map(k => (k, false)), true)
// Shuffle the table (if necessary)
val shuffledTbl = partitioner match {
case None => {
if (tbl.partitioner.isEmpty) {
new ShuffledRDD[K, Boolean](tbl, Partitioner.defaultPartitioner(tbl))
// @todo: I don't need the boolean its only there to be the second type of the shuffle.
new ShuffledRDD[K, Boolean, (K, Boolean)](tbl, Partitioner.defaultPartitioner(tbl))
} else { tbl }
}
case Some(partitioner) =>

View file

@ -15,15 +15,20 @@
* limitations under the License.
*/
package org.apache.hadoop.mapreduce
package spark.rdd
import org.apache.hadoop.conf.Configuration
trait HadoopMapReduceUtil {
def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContext(conf, jobId)
import spark.{TaskContext, Partition, RDD}
def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId)
private[spark]
class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U)
extends RDD[(K, U)](prev) {
def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier,
jobId, isMap, taskId, attemptId)
override def getPartitions = firstParent[Product2[K, U]].partitions
override val partitioner = firstParent[Product2[K, U]].partitioner
override def compute(split: Partition, context: TaskContext): Iterator[(K, U)] = {
firstParent[Product2[K, V]].iterator(split, context).map { case Product2(k ,v) => (k, f(v)) }
}
}

View file

@ -43,7 +43,7 @@ class NewHadoopRDD[K, V](
valueClass: Class[V],
@transient conf: Configuration)
extends RDD[(K, V)](sc, Nil)
with HadoopMapReduceUtil
with SparkHadoopMapReduceUtil
with Logging {
// A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it

View file

@ -0,0 +1,51 @@
/*
* 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 spark.rdd
import spark.{RangePartitioner, Logging, RDD}
/**
* Extra functions available on RDDs of (key, value) pairs where the key is sortable through
* an implicit conversion. Import `spark.SparkContext._` at the top of your program to use these
* functions. They will work with any key type that has a `scala.math.Ordered` implementation.
*/
class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest,
V: ClassManifest,
P <: Product2[K, V] : ClassManifest](
self: RDD[P])
extends Logging with Serializable {
/**
* Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling
* `collect` or `save` on the resulting RDD will return or output an ordered list of records
* (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in
* order of the keys).
*/
def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[P] = {
val part = new RangePartitioner(numPartitions, self, ascending)
val shuffled = new ShuffledRDD[K, V, P](self, part)
shuffled.mapPartitions(iter => {
val buf = iter.toArray
if (ascending) {
buf.sortWith((x, y) => x._1 < y._1).iterator
} else {
buf.sortWith((x, y) => x._1 > y._1).iterator
}
}, preservesPartitioning = true)
}
}

View file

@ -17,9 +17,7 @@
package spark.rdd
import spark._
import scala.Some
import scala.Some
import spark.{Dependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext}
private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
@ -34,14 +32,14 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
* @tparam K the key class.
* @tparam V the value class.
*/
class ShuffledRDD[K, V](
@transient var prev: RDD[(K, V)],
class ShuffledRDD[K, V, P <: Product2[K, V] : ClassManifest](
@transient var prev: RDD[P],
part: Partitioner)
extends RDD[(K, V)](prev.context, Nil) {
extends RDD[P](prev.context, Nil) {
private var serializerClass: String = null
def setSerializer(cls: String): ShuffledRDD[K, V] = {
def setSerializer(cls: String): ShuffledRDD[K, V, P] = {
serializerClass = cls
this
}
@ -56,9 +54,9 @@ class ShuffledRDD[K, V](
Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i))
}
override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = {
override def compute(split: Partition, context: TaskContext): Iterator[P] = {
val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId
SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics,
SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context.taskMetrics,
SparkEnv.get.serializerManager.get(serializerClass))
}

View file

@ -47,8 +47,8 @@ import spark.OneToOneDependency
* out of memory because of the size of `rdd2`.
*/
private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassManifest](
@transient var rdd1: RDD[(K, V)],
@transient var rdd2: RDD[(K, W)],
@transient var rdd1: RDD[_ <: Product2[K, V]],
@transient var rdd2: RDD[_ <: Product2[K, W]],
part: Partitioner)
extends RDD[(K, V)](rdd1.context, Nil) {
@ -62,11 +62,11 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM
override def getDependencies: Seq[Dependency[_]] = {
Seq(rdd1, rdd2).map { rdd =>
if (rdd.partitioner == Some(part)) {
logInfo("Adding one-to-one dependency with " + rdd)
logDebug("Adding one-to-one dependency with " + rdd)
new OneToOneDependency(rdd)
} else {
logInfo("Adding shuffle dependency with " + rdd)
new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part, serializerClass)
logDebug("Adding shuffle dependency with " + rdd)
new ShuffleDependency(rdd, part, serializerClass)
}
}
}
@ -103,16 +103,14 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM
seq
}
}
def integrate(dep: CoGroupSplitDep, op: ((K, V)) => Unit) = dep match {
def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit) = dep match {
case NarrowCoGroupSplitDep(rdd, _, itsSplit) => {
for (t <- rdd.iterator(itsSplit, context))
op(t.asInstanceOf[(K, V)])
rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op)
}
case ShuffleCoGroupSplitDep(shuffleId) => {
val iter = SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index,
val iter = SparkEnv.get.shuffleFetcher.fetch[Product2[K, V]](shuffleId, partition.index,
context.taskMetrics, serializer)
for (t <- iter)
op(t.asInstanceOf[(K, V)])
iter.foreach(op)
}
}
// the first dep is rdd1; add all values to the map

View file

@ -55,27 +55,15 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest](
}
override def getPreferredLocations(s: Partition): Seq[String] = {
// Note that as number of rdd's increase and/or number of slaves in cluster increase, the computed preferredLocations below
// become diminishingly small : so we might need to look at alternate strategies to alleviate this.
// If there are no (or very small number of preferred locations), we will end up transferred the blocks to 'any' node in the
// cluster - paying with n/w and cache cost.
// Maybe pick a node which figures max amount of time ?
// Choose node which is hosting 'larger' of some subset of blocks ?
// Look at rack locality to ensure chosen host is atleast rack local to both hosting node ?, etc (would be good to defer this if possible)
val splits = s.asInstanceOf[ZippedPartitionsPartition].partitions
val rddSplitZip = rdds.zip(splits)
// exact match.
val exactMatchPreferredLocations = rddSplitZip.map(x => x._1.preferredLocations(x._2))
val exactMatchLocations = exactMatchPreferredLocations.reduce((x, y) => x.intersect(y))
// Remove exact match and then do host local match.
val exactMatchHosts = exactMatchLocations.map(Utils.parseHostPort(_)._1)
val matchPreferredHosts = exactMatchPreferredLocations.map(locs => locs.map(Utils.parseHostPort(_)._1))
.reduce((x, y) => x.intersect(y))
val otherNodeLocalLocations = matchPreferredHosts.filter { s => !exactMatchHosts.contains(s) }
otherNodeLocalLocations ++ exactMatchLocations
val parts = s.asInstanceOf[ZippedPartitionsPartition].partitions
val prefs = rdds.zip(parts).map { case (rdd, p) => rdd.preferredLocations(p) }
// Check whether there are any hosts that match all RDDs; otherwise return the union
val exactMatchLocations = prefs.reduce((x, y) => x.intersect(y))
if (!exactMatchLocations.isEmpty) {
exactMatchLocations
} else {
prefs.flatten.distinct
}
}
override def clearDependencies() {

View file

@ -65,27 +65,16 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest](
}
override def getPreferredLocations(s: Partition): Seq[String] = {
// Note that as number of slaves in cluster increase, the computed preferredLocations can become small : so we might need
// to look at alternate strategies to alleviate this. (If there are no (or very small number of preferred locations), we
// will end up transferred the blocks to 'any' node in the cluster - paying with n/w and cache cost.
// Maybe pick one or the other ? (so that atleast one block is local ?).
// Choose node which is hosting 'larger' of the blocks ?
// Look at rack locality to ensure chosen host is atleast rack local to both hosting node ?, etc (would be good to defer this if possible)
val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions
val pref1 = rdd1.preferredLocations(partition1)
val pref2 = rdd2.preferredLocations(partition2)
// exact match - instance local and host local.
// Check whether there are any hosts that match both RDDs; otherwise return the union
val exactMatchLocations = pref1.intersect(pref2)
// remove locations which are already handled via exactMatchLocations, and intersect where both partitions are node local.
val otherNodeLocalPref1 = pref1.filter(loc => ! exactMatchLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1)
val otherNodeLocalPref2 = pref2.filter(loc => ! exactMatchLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1)
val otherNodeLocalLocations = otherNodeLocalPref1.intersect(otherNodeLocalPref2)
// Can have mix of instance local (hostPort) and node local (host) locations as preference !
exactMatchLocations ++ otherNodeLocalLocations
if (!exactMatchLocations.isEmpty) {
exactMatchLocations
} else {
(pref1 ++ pref2).distinct
}
}
override def clearDependencies() {

View file

@ -25,7 +25,7 @@ import java.util.Properties
* Tracks information about an active job in the DAGScheduler.
*/
private[spark] class ActiveJob(
val runId: Int,
val jobId: Int,
val finalStage: Stage,
val func: (TaskContext, Iterator[_]) => _,
val partitions: Array[Int],

View file

@ -32,10 +32,22 @@ import spark.storage.{BlockManager, BlockManagerMaster}
import spark.util.{MetadataCleaner, TimeStampedHashMap}
/**
* A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for
* each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal
* schedule to run the job. Subclasses only need to implement the code to send a task to the cluster
* and to report fetch failures (the submitTasks method, and code to add CompletionEvents).
* The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of
* stages for each job, keeps track of which RDDs and stage outputs are materialized, and finds a
* minimal schedule to run the job. It then submits stages as TaskSets to an underlying
* TaskScheduler implementation that runs them on the cluster.
*
* In addition to coming up with a DAG of stages, this class also determines the preferred
* locations to run each task on, based on the current cache status, and passes these to the
* low-level TaskScheduler. Furthermore, it handles failures due to shuffle output files being
* lost, in which case old stages may need to be resubmitted. Failures *within* a stage that are
* not caused by shuffie file loss are handled by the TaskScheduler, which will retry each task
* a small number of times before cancelling the whole stage.
*
* THREADING: This class runs all its logic in a single thread executing the run() method, to which
* events are submitted using a synchonized queue (eventQueue). The public API methods, such as
* runJob, taskEnded and executorLost, post events asynchronously to this queue. All other methods
* should be private.
*/
private[spark]
class DAGScheduler(
@ -72,8 +84,8 @@ class DAGScheduler(
}
// Called by TaskScheduler when a host is added
override def executorGained(execId: String, hostPort: String) {
eventQueue.put(ExecutorGained(execId, hostPort))
override def executorGained(execId: String, host: String) {
eventQueue.put(ExecutorGained(execId, host))
}
// Called by TaskScheduler to cancel an entire TaskSet due to repeated failures.
@ -92,11 +104,11 @@ class DAGScheduler(
private val eventQueue = new LinkedBlockingQueue[DAGSchedulerEvent]
val nextRunId = new AtomicInteger(0)
val nextJobId = new AtomicInteger(0)
val nextStageId = new AtomicInteger(0)
val idToStage = new TimeStampedHashMap[Int, Stage]
val stageIdToStage = new TimeStampedHashMap[Int, Stage]
val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
@ -104,15 +116,16 @@ class DAGScheduler(
private val listenerBus = new SparkListenerBus()
var cacheLocs = new HashMap[Int, Array[List[String]]]
// Contains the locations that each RDD's partitions are cached on
private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]]
// For tracking failed nodes, we use the MapOutputTracker's generation number, which is
// sent with every task. When we detect a node failing, we note the current generation number
// and failed executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask
// results.
// TODO: Garbage collect information about failure generations when we know there are no more
// For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with
// every task. When we detect a node failing, we note the current epoch number and failed
// executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask results.
//
// TODO: Garbage collect information about failure epochs when we know there are no more
// stray messages to detect.
val failedGeneration = new HashMap[String, Long]
val failedEpoch = new HashMap[String, Long]
val idToActiveJob = new HashMap[Int, ActiveJob]
@ -141,11 +154,13 @@ class DAGScheduler(
listenerBus.addListener(listener)
}
private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = {
private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = {
if (!cacheLocs.contains(rdd.id)) {
val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray
val locs = BlockManager.blockIdsToExecutorLocations(blockIds, env, blockManagerMaster)
cacheLocs(rdd.id) = blockIds.map(locs.getOrElse(_, Nil))
val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster)
cacheLocs(rdd.id) = blockIds.map { id =>
locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId))
}
}
cacheLocs(rdd.id)
}
@ -156,14 +171,14 @@ class DAGScheduler(
/**
* Get or create a shuffle map stage for the given shuffle dependency's map side.
* The priority value passed in will be used if the stage doesn't already exist with
* a lower priority (we assume that priorities always increase across jobs for now).
* The jobId value passed in will be used if the stage doesn't already exist with
* a lower jobId (jobId always increases across jobs.)
*/
private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int): Stage = {
private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], jobId: Int): Stage = {
shuffleToMapStage.get(shuffleDep.shuffleId) match {
case Some(stage) => stage
case None =>
val stage = newStage(shuffleDep.rdd, Some(shuffleDep), priority)
val stage = newStage(shuffleDep.rdd, Some(shuffleDep), jobId)
shuffleToMapStage(shuffleDep.shuffleId) = stage
stage
}
@ -171,13 +186,13 @@ class DAGScheduler(
/**
* Create a Stage for the given RDD, either as a shuffle map stage (for a ShuffleDependency) or
* as a result stage for the final RDD used directly in an action. The stage will also be given
* the provided priority.
* as a result stage for the final RDD used directly in an action. The stage will also be
* associated with the provided jobId.
*/
private def newStage(
rdd: RDD[_],
shuffleDep: Option[ShuffleDependency[_,_]],
priority: Int,
jobId: Int,
callSite: Option[String] = None)
: Stage =
{
@ -188,17 +203,17 @@ class DAGScheduler(
mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size)
}
val id = nextStageId.getAndIncrement()
val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority, callSite)
idToStage(id) = stage
val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, jobId), jobId, callSite)
stageIdToStage(id) = stage
stageToInfos(stage) = StageInfo(stage)
stage
}
/**
* Get or create the list of parent stages for a given RDD. The stages will be assigned the
* provided priority if they haven't already been created with a lower priority.
* provided jobId if they haven't already been created with a lower jobId.
*/
private def getParentStages(rdd: RDD[_], priority: Int): List[Stage] = {
private def getParentStages(rdd: RDD[_], jobId: Int): List[Stage] = {
val parents = new HashSet[Stage]
val visited = new HashSet[RDD[_]]
def visit(r: RDD[_]) {
@ -209,7 +224,7 @@ class DAGScheduler(
for (dep <- r.dependencies) {
dep match {
case shufDep: ShuffleDependency[_,_] =>
parents += getShuffleMapStage(shufDep, priority)
parents += getShuffleMapStage(shufDep, jobId)
case _ =>
visit(dep.rdd)
}
@ -230,7 +245,7 @@ class DAGScheduler(
for (dep <- rdd.dependencies) {
dep match {
case shufDep: ShuffleDependency[_,_] =>
val mapStage = getShuffleMapStage(shufDep, stage.priority)
val mapStage = getShuffleMapStage(shufDep, stage.jobId)
if (!mapStage.isAvailable) {
missing += mapStage
}
@ -267,7 +282,7 @@ class DAGScheduler(
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter,
properties)
return (toSubmit, waiter)
(toSubmit, waiter)
}
def runJob[T, U: ClassManifest](
@ -314,8 +329,8 @@ class DAGScheduler(
val listener = new ApproximateActionListener(rdd, func, evaluator, timeout)
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
val partitions = (0 until rdd.partitions.size).toArray
eventQueue.put(JobSubmitted(rdd, func2, partitions, false, callSite, listener, properties))
return listener.awaitResult() // Will throw an exception if the job fails
eventQueue.put(JobSubmitted(rdd, func2, partitions, allowLocal = false, callSite, listener, properties))
listener.awaitResult() // Will throw an exception if the job fails
}
/**
@ -325,11 +340,11 @@ class DAGScheduler(
private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = {
event match {
case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) =>
val runId = nextRunId.getAndIncrement()
val finalStage = newStage(finalRDD, None, runId, Some(callSite))
val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties)
val jobId = nextJobId.getAndIncrement()
val finalStage = newStage(finalRDD, None, jobId, Some(callSite))
val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties)
clearCacheLocs()
logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length +
logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length +
" output partitions (allowLocal=" + allowLocal + ")")
logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")")
logInfo("Parents of final stage: " + finalStage.parents)
@ -339,14 +354,14 @@ class DAGScheduler(
runLocally(job)
} else {
listenerBus.post(SparkListenerJobStart(job, properties))
idToActiveJob(runId) = job
idToActiveJob(jobId) = job
activeJobs += job
resultStageToJob(finalStage) = job
submitStage(finalStage)
}
case ExecutorGained(execId, hostPort) =>
handleExecutorGained(execId, hostPort)
case ExecutorGained(execId, host) =>
handleExecutorGained(execId, host)
case ExecutorLost(execId) =>
handleExecutorLost(execId)
@ -360,7 +375,7 @@ class DAGScheduler(
handleTaskCompletion(completion)
case TaskSetFailed(taskSet, reason) =>
abortStage(idToStage(taskSet.stageId), reason)
abortStage(stageIdToStage(taskSet.stageId), reason)
case StopDAGScheduler =>
// Cancel any active jobs
@ -371,7 +386,7 @@ class DAGScheduler(
}
return true
}
return false
false
}
/**
@ -383,7 +398,7 @@ class DAGScheduler(
clearCacheLocs()
val failed2 = failed.toArray
failed.clear()
for (stage <- failed2.sortBy(_.priority)) {
for (stage <- failed2.sortBy(_.jobId)) {
submitStage(stage)
}
}
@ -401,7 +416,7 @@ class DAGScheduler(
logTrace("failed: " + failed)
val waiting2 = waiting.toArray
waiting.clear()
for (stage <- waiting2.sortBy(_.priority)) {
for (stage <- waiting2.sortBy(_.jobId)) {
submitStage(stage)
}
}
@ -420,7 +435,7 @@ class DAGScheduler(
if (event != null) {
logDebug("Got event of type " + event.getClass.getName)
}
this.synchronized { // needed in case other threads makes calls into methods of this class
if (event != null) {
if (processEvent(event)) {
return
@ -440,6 +455,7 @@ class DAGScheduler(
}
}
}
}
/**
* Run a job on an RDD locally, assuming it has only a single partition and no dependencies.
@ -448,7 +464,7 @@ class DAGScheduler(
*/
protected def runLocally(job: ActiveJob) {
logInfo("Computing the requested partition locally")
new Thread("Local computation of job " + job.runId) {
new Thread("Local computation of job " + job.jobId) {
override def run() {
runLocallyWithinThread(job)
}
@ -508,7 +524,7 @@ class DAGScheduler(
} else {
// This is a final stage; figure out its job's missing partitions
val job = resultStageToJob(stage)
for (id <- 0 until job.numPartitions if (!job.finished(id))) {
for (id <- 0 until job.numPartitions if !job.finished(id)) {
val partition = job.partitions(id)
val locs = getPreferredLocs(stage.rdd, partition)
tasks += new ResultTask(stage.id, stage.rdd, job.func, partition, locs, id)
@ -516,7 +532,7 @@ class DAGScheduler(
}
// must be run listener before possible NotSerializableException
// should be "StageSubmitted" first and then "JobEnded"
val properties = idToActiveJob(stage.priority).properties
val properties = idToActiveJob(stage.jobId).properties
listenerBus.post(SparkListenerStageSubmitted(stage, tasks.size, properties))
if (tasks.size > 0) {
@ -537,7 +553,7 @@ class DAGScheduler(
myPending ++= tasks
logDebug("New pending tasks: " + myPending)
taskSched.submitTasks(
new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority, properties))
new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties))
if (!stage.submissionTime.isDefined) {
stage.submissionTime = Some(System.currentTimeMillis())
}
@ -554,7 +570,7 @@ class DAGScheduler(
*/
private def handleTaskCompletion(event: CompletionEvent) {
val task = event.task
val stage = idToStage(task.stageId)
val stage = stageIdToStage(task.stageId)
def markStageAsFinished(stage: Stage) = {
val serviceTime = stage.submissionTime match {
@ -583,7 +599,7 @@ class DAGScheduler(
job.numFinished += 1
// If the whole job has finished, remove it
if (job.numFinished == job.numPartitions) {
idToActiveJob -= stage.priority
idToActiveJob -= stage.jobId
activeJobs -= job
resultStageToJob -= stage
markStageAsFinished(stage)
@ -599,7 +615,7 @@ class DAGScheduler(
val status = event.result.asInstanceOf[MapStatus]
val execId = status.location.executorId
logDebug("ShuffleMapTask finished on " + execId)
if (failedGeneration.contains(execId) && smt.generation <= failedGeneration(execId)) {
if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) {
logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId)
} else {
stage.addOutputLoc(smt.partition, status)
@ -611,16 +627,16 @@ class DAGScheduler(
logInfo("waiting: " + waiting)
logInfo("failed: " + failed)
if (stage.shuffleDep != None) {
// We supply true to increment the generation number here in case this is a
// We supply true to increment the epoch number here in case this is a
// recomputation of the map outputs. In that case, some nodes may have cached
// locations with holes (from when we detected the error) and will need the
// generation incremented to refetch them.
// TODO: Only increment the generation number if this is not the first time
// epoch incremented to refetch them.
// TODO: Only increment the epoch number if this is not the first time
// we registered these map outputs.
mapOutputTracker.registerMapOutputs(
stage.shuffleDep.get.shuffleId,
stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray,
true)
changeEpoch = true)
}
clearCacheLocs()
if (stage.outputLocs.count(_ == Nil) != 0) {
@ -654,7 +670,7 @@ class DAGScheduler(
case FetchFailed(bmAddress, shuffleId, mapId, reduceId) =>
// Mark the stage that the reducer was in as unrunnable
val failedStage = idToStage(task.stageId)
val failedStage = stageIdToStage(task.stageId)
running -= failedStage
failed += failedStage
// TODO: Cancel running tasks in the stage
@ -674,7 +690,7 @@ class DAGScheduler(
lastFetchFailureTime = System.currentTimeMillis() // TODO: Use pluggable clock
// TODO: mark the executor as failed only if there were lots of fetch failures on it
if (bmAddress != null) {
handleExecutorLost(bmAddress.executorId, Some(task.generation))
handleExecutorLost(bmAddress.executorId, Some(task.epoch))
}
case ExceptionFailure(className, description, stackTrace, metrics) =>
@ -682,7 +698,7 @@ class DAGScheduler(
case other =>
// Unrecognized failure - abort all jobs depending on this stage
abortStage(idToStage(task.stageId), task + " failed: " + other)
abortStage(stageIdToStage(task.stageId), task + " failed: " + other)
}
}
@ -690,36 +706,36 @@ class DAGScheduler(
* Responds to an executor being lost. This is called inside the event loop, so it assumes it can
* modify the scheduler's internal state. Use executorLost() to post a loss event from outside.
*
* Optionally the generation during which the failure was caught can be passed to avoid allowing
* Optionally the epoch during which the failure was caught can be passed to avoid allowing
* stray fetch failures from possibly retriggering the detection of a node as lost.
*/
private def handleExecutorLost(execId: String, maybeGeneration: Option[Long] = None) {
val currentGeneration = maybeGeneration.getOrElse(mapOutputTracker.getGeneration)
if (!failedGeneration.contains(execId) || failedGeneration(execId) < currentGeneration) {
failedGeneration(execId) = currentGeneration
logInfo("Executor lost: %s (generation %d)".format(execId, currentGeneration))
private def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) {
val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch)
if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) {
failedEpoch(execId) = currentEpoch
logInfo("Executor lost: %s (epoch %d)".format(execId, currentEpoch))
blockManagerMaster.removeExecutor(execId)
// TODO: This will be really slow if we keep accumulating shuffle map stages
for ((shuffleId, stage) <- shuffleToMapStage) {
stage.removeOutputsOnExecutor(execId)
val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray
mapOutputTracker.registerMapOutputs(shuffleId, locs, true)
mapOutputTracker.registerMapOutputs(shuffleId, locs, changeEpoch = true)
}
if (shuffleToMapStage.isEmpty) {
mapOutputTracker.incrementGeneration()
mapOutputTracker.incrementEpoch()
}
clearCacheLocs()
} else {
logDebug("Additional executor lost message for " + execId +
"(generation " + currentGeneration + ")")
"(epoch " + currentEpoch + ")")
}
}
private def handleExecutorGained(execId: String, hostPort: String) {
// remove from failedGeneration(execId) ?
if (failedGeneration.contains(execId)) {
logInfo("Host gained which was in lost list earlier: " + hostPort)
failedGeneration -= execId
private def handleExecutorGained(execId: String, host: String) {
// remove from failedEpoch(execId) ?
if (failedEpoch.contains(execId)) {
logInfo("Host gained which was in lost list earlier: " + host)
failedEpoch -= execId
}
}
@ -735,7 +751,7 @@ class DAGScheduler(
val error = new SparkException("Job failed: " + reason)
job.listener.jobFailed(error)
listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage))))
idToActiveJob -= resultStage.priority
idToActiveJob -= resultStage.jobId
activeJobs -= job
resultStageToJob -= resultStage
}
@ -759,7 +775,7 @@ class DAGScheduler(
for (dep <- rdd.dependencies) {
dep match {
case shufDep: ShuffleDependency[_,_] =>
val mapStage = getShuffleMapStage(shufDep, stage.priority)
val mapStage = getShuffleMapStage(shufDep, stage.jobId)
if (!mapStage.isAvailable) {
visitedStages += mapStage
visit(mapStage.rdd)
@ -774,16 +790,23 @@ class DAGScheduler(
visitedRdds.contains(target.rdd)
}
private def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = {
/**
* Synchronized method that might be called from other threads.
* @param rdd whose partitions are to be looked at
* @param partition to lookup locality information for
* @return list of machines that are preferred by the partition
*/
private[spark]
def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = synchronized {
// If the partition is cached, return the cache locations
val cached = getCacheLocs(rdd)(partition)
if (cached != Nil) {
if (!cached.isEmpty) {
return cached
}
// If the RDD has some placement preferences (as is the case for input RDDs), get those
val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList
if (rddPrefs != Nil) {
return rddPrefs
if (!rddPrefs.isEmpty) {
return rddPrefs.map(host => TaskLocation(host))
}
// If the RDD has narrow dependencies, pick the first partition of the first narrow dep
// that has any placement preferences. Ideally we would choose based on transfer sizes,
@ -797,13 +820,13 @@ class DAGScheduler(
}
case _ =>
})
return Nil
Nil
}
private def cleanup(cleanupTime: Long) {
var sizeBefore = idToStage.size
idToStage.clearOldValues(cleanupTime)
logInfo("idToStage " + sizeBefore + " --> " + idToStage.size)
var sizeBefore = stageIdToStage.size
stageIdToStage.clearOldValues(cleanupTime)
logInfo("stageIdToStage " + sizeBefore + " --> " + stageIdToStage.size)
sizeBefore = shuffleToMapStage.size
shuffleToMapStage.clearOldValues(cleanupTime)

View file

@ -54,9 +54,7 @@ private[spark] case class CompletionEvent(
taskMetrics: TaskMetrics)
extends DAGSchedulerEvent
private[spark] case class ExecutorGained(execId: String, hostPort: String) extends DAGSchedulerEvent {
Utils.checkHostPort(hostPort, "Required hostport")
}
private[spark] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent
private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent

View file

@ -21,7 +21,7 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends
})
metricRegistry.register(MetricRegistry.name("job", "allJobs", "number"), new Gauge[Int] {
override def getValue: Int = dagScheduler.nextRunId.get()
override def getValue: Int = dagScheduler.nextJobId.get()
})
metricRegistry.register(MetricRegistry.name("job", "activeJobs", "number"), new Gauge[Int] {

View file

@ -17,7 +17,7 @@
package spark.scheduler
import spark.Logging
import spark.{Logging, SparkEnv}
import scala.collection.immutable.Set
import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
import org.apache.hadoop.security.UserGroupInformation
@ -26,7 +26,6 @@ import org.apache.hadoop.mapreduce.Job
import org.apache.hadoop.conf.Configuration
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
import scala.collection.JavaConversions._
import spark.deploy.SparkHadoopUtil
/**
@ -88,8 +87,9 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
// This method does not expect failures, since validate has already passed ...
private def prefLocsFromMapreduceInputFormat(): Set[SplitInfo] = {
val env = SparkEnv.get
val conf = new JobConf(configuration)
SparkHadoopUtil.addCredentials(conf);
env.hadoop.addCredentials(conf)
FileInputFormat.setInputPaths(conf, path)
val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] =
@ -108,8 +108,9 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
// This method does not expect failures, since validate has already passed ...
private def prefLocsFromMapredInputFormat(): Set[SplitInfo] = {
val env = SparkEnv.get
val jobConf = new JobConf(configuration)
SparkHadoopUtil.addCredentials(jobConf);
env.hadoop.addCredentials(jobConf)
FileInputFormat.setInputPaths(jobConf, path)
val instance: org.apache.hadoop.mapred.InputFormat[_, _] =

View file

@ -102,7 +102,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging {
stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime))
protected def buildJobDep(jobID: Int, stage: Stage) {
if (stage.priority == jobID) {
if (stage.jobId == jobID) {
jobIDToStages.get(jobID) match {
case Some(stageList) => stageList += stage
case None => val stageList = new ListBuffer[Stage]
@ -178,12 +178,12 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging {
}else{
stageInfo = "STAGE_ID=" + stage.id + " RESULT_STAGE"
}
if (stage.priority == jobID) {
if (stage.jobId == jobID) {
jobLogInfo(jobID, indentString(indent) + stageInfo, false)
recordRddInStageGraph(jobID, stage.rdd, indent)
stage.parents.foreach(recordStageDepGraph(jobID, _, indent + 2))
} else
jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.priority, false)
jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.jobId, false)
}
// Record task metrics into job log files
@ -260,7 +260,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging {
override def onJobEnd(jobEnd: SparkListenerJobEnd) {
val job = jobEnd.job
var info = "JOB_ID=" + job.runId
var info = "JOB_ID=" + job.jobId
jobEnd.jobResult match {
case JobSucceeded => info += " STATUS=SUCCESS"
case JobFailed(exception, _) =>
@ -268,8 +268,8 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging {
exception.getMessage.split("\\s+").foreach(info += _ + "_")
case _ =>
}
jobLogInfo(job.runId, info.substring(0, info.length - 1).toUpperCase)
closeLogWriter(job.runId)
jobLogInfo(job.jobId, info.substring(0, info.length - 1).toUpperCase)
closeLogWriter(job.jobId)
}
protected def recordJobProperties(jobID: Int, properties: Properties) {
@ -282,11 +282,11 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging {
override def onJobStart(jobStart: SparkListenerJobStart) {
val job = jobStart.job
val properties = jobStart.properties
createLogWriter(job.runId)
recordJobProperties(job.runId, properties)
buildJobDep(job.runId, job.finalStage)
recordStageDep(job.runId)
recordStageDepGraph(job.runId, job.finalStage)
jobLogInfo(job.runId, "JOB_ID=" + job.runId + " STATUS=STARTED")
createLogWriter(job.jobId)
recordJobProperties(job.jobId, properties)
buildJobDep(job.jobId, job.finalStage)
recordStageDep(job.jobId)
recordStageDepGraph(job.jobId, job.finalStage)
jobLogInfo(job.jobId, "JOB_ID=" + job.jobId + " STATUS=STARTED")
}
}

View file

@ -73,7 +73,7 @@ private[spark] class ResultTask[T, U](
var rdd: RDD[T],
var func: (TaskContext, Iterator[T]) => U,
var partition: Int,
@transient locs: Seq[String],
@transient locs: Seq[TaskLocation],
val outputId: Int)
extends Task[U](stageId) with Externalizable {
@ -85,11 +85,8 @@ private[spark] class ResultTask[T, U](
rdd.partitions(partition)
}
private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq
{
// DEBUG code
preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs))
@transient private val preferredLocs: Seq[TaskLocation] = {
if (locs == null) Nil else locs.toSet.toSeq
}
override def run(attemptId: Long): U = {
@ -102,7 +99,7 @@ private[spark] class ResultTask[T, U](
}
}
override def preferredLocations: Seq[String] = preferredLocs
override def preferredLocations: Seq[TaskLocation] = preferredLocs
override def toString = "ResultTask(" + stageId + ", " + partition + ")"
@ -116,7 +113,7 @@ private[spark] class ResultTask[T, U](
out.write(bytes)
out.writeInt(partition)
out.writeInt(outputId)
out.writeLong(generation)
out.writeLong(epoch)
out.writeObject(split)
}
}
@ -131,7 +128,7 @@ private[spark] class ResultTask[T, U](
func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U]
partition = in.readInt()
val outputId = in.readInt()
generation = in.readLong()
epoch = in.readLong()
split = in.readObject().asInstanceOf[Partition]
}
}

View file

@ -88,18 +88,15 @@ private[spark] class ShuffleMapTask(
var rdd: RDD[_],
var dep: ShuffleDependency[_,_],
var partition: Int,
@transient private var locs: Seq[String])
@transient private var locs: Seq[TaskLocation])
extends Task[MapStatus](stageId)
with Externalizable
with Logging {
protected def this() = this(0, null, null, 0, null)
@transient private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq
{
// DEBUG code
preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs))
@transient private val preferredLocs: Seq[TaskLocation] = {
if (locs == null) Nil else locs.toSet.toSeq
}
var split = if (rdd == null) null else rdd.partitions(partition)
@ -112,7 +109,7 @@ private[spark] class ShuffleMapTask(
out.writeInt(bytes.length)
out.write(bytes)
out.writeInt(partition)
out.writeLong(generation)
out.writeLong(epoch)
out.writeObject(split)
}
}
@ -126,7 +123,7 @@ private[spark] class ShuffleMapTask(
rdd = rdd_
dep = dep_
partition = in.readInt()
generation = in.readLong()
epoch = in.readLong()
split = in.readObject().asInstanceOf[Partition]
}
@ -148,7 +145,7 @@ private[spark] class ShuffleMapTask(
// Write the map output to its associated buckets.
for (elem <- rdd.iterator(split, taskContext)) {
val pair = elem.asInstanceOf[(Any, Any)]
val pair = elem.asInstanceOf[Product2[Any, Any]]
val bucketId = dep.partitioner.getPartition(pair._1)
buckets.writers(bucketId).write(pair)
}
@ -186,7 +183,7 @@ private[spark] class ShuffleMapTask(
}
}
override def preferredLocations: Seq[String] = preferredLocs
override def preferredLocations: Seq[TaskLocation] = preferredLocs
override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partition)
}

View file

@ -33,15 +33,16 @@ import spark.storage.BlockManagerId
* initiated a job (e.g. count(), save(), etc). For shuffle map stages, we also track the nodes
* that each output partition is on.
*
* Each Stage also has a priority, which is (by default) based on the job it was submitted in.
* This allows Stages from earlier jobs to be computed first or recovered faster on failure.
* Each Stage also has a jobId, identifying the job that first submitted the stage. When FIFO
* scheduling is used, this allows Stages from earlier jobs to be computed first or recovered
* faster on failure.
*/
private[spark] class Stage(
val id: Int,
val rdd: RDD[_],
val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage
val parents: List[Stage],
val priority: Int,
val jobId: Int,
callSite: Option[String])
extends Logging {

View file

@ -30,9 +30,9 @@ import spark.executor.TaskMetrics
*/
private[spark] abstract class Task[T](val stageId: Int) extends Serializable {
def run(attemptId: Long): T
def preferredLocations: Seq[String] = Nil
def preferredLocations: Seq[TaskLocation] = Nil
var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler.
var epoch: Long = -1 // Map output tracker epoch. Will be set by TaskScheduler.
var metrics: Option[TaskMetrics] = None

View file

@ -15,16 +15,20 @@
* limitations under the License.
*/
package org.apache.hadoop.mapreduce
package spark.scheduler
import org.apache.hadoop.conf.Configuration
import task.{TaskAttemptContextImpl, JobContextImpl}
trait HadoopMapReduceUtil {
def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) =
new TaskAttemptID(jtIdentifier, jobId, if (isMap) TaskType.MAP else TaskType.REDUCE, taskId, attemptId)
/**
* A location where a task should run. This can either be a host or a (host, executorID) pair.
* In the latter case, we will prefer to launch the task on that executorID, but our next level
* of preference will be executors on the same host if this is not possible.
*/
private[spark]
class TaskLocation private (val host: String, val executorId: Option[String]) extends Serializable {
override def toString: String = "TaskLocation(" + host + ", " + executorId + ")"
}
private[spark] object TaskLocation {
def apply(host: String, executorId: String) = new TaskLocation(host, Some(executorId))
def apply(host: String) = new TaskLocation(host, None)
}

View file

@ -28,7 +28,9 @@ import java.nio.ByteBuffer
// TODO: Use of distributed cache to return result is a hack to get around
// what seems to be a bug with messages over 60KB in libprocess; fix it
private[spark]
class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) extends Externalizable {
class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics)
extends Externalizable
{
def this() = this(null.asInstanceOf[T], null, null)
override def writeExternal(out: ObjectOutput) {

View file

@ -35,7 +35,7 @@ private[spark] trait TaskSchedulerListener {
taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit
// A node was added to the cluster.
def executorGained(execId: String, hostPort: String): Unit
def executorGained(execId: String, host: String): Unit
// A node was lost from the cluster.
def executorLost(execId: String): Unit

View file

@ -33,45 +33,27 @@ import java.util.{TimerTask, Timer}
/**
* The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call
* start(), then submit task sets through the runTasks method.
* initialize() and start(), then submit task sets through the runTasks method.
*
* This class can work with multiple types of clusters by acting through a SchedulerBackend.
* It handles common logic, like determining a scheduling order across jobs, waking up to launch
* speculative tasks, etc.
*
* THREADING: SchedulerBackends and task-submitting clients can call this class from multiple
* threads, so it needs locks in public API methods to maintain its state. In addition, some
* SchedulerBackends sycnchronize on themselves when they want to send events here, and then
* acquire a lock on us, so we need to make sure that we don't try to lock the backend while
* we are holding a lock on ourselves.
*/
private[spark] class ClusterScheduler(val sc: SparkContext)
extends TaskScheduler
with Logging {
with Logging
{
// How often to check for speculative tasks
val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong
// Threshold above which we warn user initial TaskSet may be starved
val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong
// How often to revive offers in case there are pending tasks - that is how often to try to get
// tasks scheduled in case there are nodes available : default 0 is to disable it - to preserve existing behavior
// Note that this is required due to delayed scheduling due to data locality waits, etc.
// TODO: rename property ?
val TASK_REVIVAL_INTERVAL = System.getProperty("spark.tasks.revive.interval", "0").toLong
/*
This property controls how aggressive we should be to modulate waiting for node local task scheduling.
To elaborate, currently there is a time limit (3 sec def) to ensure that spark attempts to wait for node locality of tasks before
scheduling on other nodes. We have modified this in yarn branch such that offers to task set happen in prioritized order :
node-local, rack-local and then others
But once all available node local (and no pref) tasks are scheduled, instead of waiting for 3 sec before
scheduling to other nodes (which degrades performance for time sensitive tasks and on larger clusters), we can
modulate that : to also allow rack local nodes or any node. The default is still set to HOST - so that previous behavior is
maintained. This is to allow tuning the tension between pulling rdd data off node and scheduling computation asap.
TODO: rename property ? The value is one of
- NODE_LOCAL (default, no change w.r.t current behavior),
- RACK_LOCAL and
- ANY
Note that this property makes more sense when used in conjugation with spark.tasks.revive.interval > 0 : else it is not very effective.
Additional Note: For non trivial clusters, there is a 4x - 5x reduction in running time (in some of our experiments) based on whether
it is left at default NODE_LOCAL, RACK_LOCAL (if cluster is configured to be rack aware) or ANY.
If cluster is rack aware, then setting it to RACK_LOCAL gives best tradeoff and a 3x - 4x performance improvement while minimizing IO impact.
Also, it brings down the variance in running time drastically.
*/
val TASK_SCHEDULING_AGGRESSION = TaskLocality.parse(System.getProperty("spark.tasks.schedule.aggression", "NODE_LOCAL"))
val activeTaskSets = new HashMap[String, TaskSetManager]
@ -89,16 +71,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
// Which executor IDs we have executors on
val activeExecutorIds = new HashSet[String]
// TODO: We might want to remove this and merge it with execId datastructures - but later.
// Which hosts in the cluster are alive (contains hostPort's) - used for process local and node local task locality.
private val hostPortsAlive = new HashSet[String]
private val hostToAliveHostPorts = new HashMap[String, HashSet[String]]
// The set of executors we have on each host; this is used to compute hostsAlive, which
// in turn is used to decide when we can attain data locality on a given host
private val executorsByHostPort = new HashMap[String, HashSet[String]]
private val executorsByHost = new HashMap[String, HashSet[String]]
private val executorIdToHostPort = new HashMap[String, String]
private val executorIdToHost = new HashMap[String, String]
// JAR server, if any JARs were added by the user to the SparkContext
var jarServer: HttpServer = null
@ -136,23 +113,14 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
}
schedulableBuilder.buildPools()
// resolve executorId to hostPort mapping.
def executorToHostPort(executorId: String, defaultHostPort: String): String = {
executorIdToHostPort.getOrElse(executorId, defaultHostPort)
}
// Unfortunately, this means that SparkEnv is indirectly referencing ClusterScheduler
// Will that be a design violation ?
SparkEnv.get.executorIdToHostPort = Some(executorToHostPort)
}
def newTaskId(): Long = nextTaskId.getAndIncrement()
override def start() {
backend.start()
if (JBoolean.getBoolean("spark.speculation")) {
if (System.getProperty("spark.speculation", "false").toBoolean) {
new Thread("ClusterScheduler speculation check") {
setDaemon(true)
@ -169,27 +137,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
}.start()
}
// Change to always run with some default if TASK_REVIVAL_INTERVAL <= 0 ?
if (TASK_REVIVAL_INTERVAL > 0) {
new Thread("ClusterScheduler task offer revival check") {
setDaemon(true)
override def run() {
logInfo("Starting speculative task offer revival thread")
while (true) {
try {
Thread.sleep(TASK_REVIVAL_INTERVAL)
} catch {
case e: InterruptedException => {}
}
if (hasPendingTasks()) backend.reviveOffers()
}
}
}.start()
}
}
override def submitTasks(taskSet: TaskSet) {
@ -201,7 +148,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties)
taskSetTaskIds(taskSet.id) = new HashSet[Long]()
if (hasReceivedTask == false) {
if (!hasReceivedTask) {
starvationTimer.scheduleAtFixedRate(new TimerTask() {
override def run() {
if (!hasLaunchedTask) {
@ -214,7 +161,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
}, STARVATION_TIMEOUT, STARVATION_TIMEOUT)
}
hasReceivedTask = true;
hasReceivedTask = true
}
backend.reviveOffers()
}
@ -235,162 +182,46 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
* 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.
*/
def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = {
synchronized {
def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized {
SparkEnv.set(sc.env)
// Mark each slave as alive and remember its hostname
for (o <- offers) {
// DEBUG Code
Utils.checkHostPort(o.hostPort)
executorIdToHostPort(o.executorId) = o.hostPort
if (! executorsByHostPort.contains(o.hostPort)) {
executorsByHostPort(o.hostPort) = new HashSet[String]()
executorIdToHost(o.executorId) = o.host
if (!executorsByHost.contains(o.host)) {
executorsByHost(o.host) = new HashSet[String]()
executorGained(o.executorId, o.host)
}
}
hostPortsAlive += o.hostPort
hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(o.hostPort)._1, new HashSet[String]).add(o.hostPort)
executorGained(o.executorId, o.hostPort)
}
// Build a list of tasks to assign to each slave
// Build a list of tasks to assign to each worker
val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores))
// merge availableCpus into nodeToAvailableCpus block ?
val availableCpus = offers.map(o => o.cores).toArray
val nodeToAvailableCpus = {
val map = new HashMap[String, Int]()
for (offer <- offers) {
val hostPort = offer.hostPort
val cores = offer.cores
// DEBUG code
Utils.checkHostPort(hostPort)
val host = Utils.parseHostPort(hostPort)._1
map.put(host, map.getOrElse(host, 0) + cores)
val sortedTaskSets = rootPool.getSortedTaskSetQueue()
for (taskSet <- sortedTaskSets) {
logDebug("parentName: %s, name: %s, runningTasks: %s".format(
taskSet.parent.name, taskSet.name, taskSet.runningTasks))
}
map
}
// Take each TaskSet in our scheduling order, and then offer it each node in increasing order
// of locality levels so that it gets a chance to launch local tasks on all of them.
var launchedTask = false
val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue()
for (manager <- sortedTaskSetQueue) {
logDebug("parentName:%s, name:%s, runningTasks:%s".format(
manager.parent.name, manager.name, manager.runningTasks))
}
for (manager <- sortedTaskSetQueue) {
// Split offers based on node local, rack local and off-rack tasks.
val processLocalOffers = new HashMap[String, ArrayBuffer[Int]]()
val nodeLocalOffers = new HashMap[String, ArrayBuffer[Int]]()
val rackLocalOffers = new HashMap[String, ArrayBuffer[Int]]()
val otherOffers = new HashMap[String, ArrayBuffer[Int]]()
for (i <- 0 until offers.size) {
val hostPort = offers(i).hostPort
// DEBUG code
Utils.checkHostPort(hostPort)
val numProcessLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i)))
if (numProcessLocalTasks > 0){
val list = processLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int])
for (j <- 0 until numProcessLocalTasks) list += i
}
val host = Utils.parseHostPort(hostPort)._1
val numNodeLocalTasks = math.max(0,
// Remove process local tasks (which are also host local btw !) from this
math.min(manager.numPendingTasksForHost(hostPort) - numProcessLocalTasks, nodeToAvailableCpus(host)))
if (numNodeLocalTasks > 0){
val list = nodeLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int])
for (j <- 0 until numNodeLocalTasks) list += i
}
val numRackLocalTasks = math.max(0,
// Remove node local tasks (which are also rack local btw !) from this
math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numProcessLocalTasks - numNodeLocalTasks, nodeToAvailableCpus(host)))
if (numRackLocalTasks > 0){
val list = rackLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int])
for (j <- 0 until numRackLocalTasks) list += i
}
if (numNodeLocalTasks <= 0 && numRackLocalTasks <= 0){
// add to others list - spread even this across cluster.
val list = otherOffers.getOrElseUpdate(host, new ArrayBuffer[Int])
list += i
}
}
val offersPriorityList = new ArrayBuffer[Int](
processLocalOffers.size + nodeLocalOffers.size + rackLocalOffers.size + otherOffers.size)
// First process local, then host local, then rack, then others
// numNodeLocalOffers contains count of both process local and host offers.
val numNodeLocalOffers = {
val processLocalPriorityList = ClusterScheduler.prioritizeContainers(processLocalOffers)
offersPriorityList ++= processLocalPriorityList
val nodeLocalPriorityList = ClusterScheduler.prioritizeContainers(nodeLocalOffers)
offersPriorityList ++= nodeLocalPriorityList
processLocalPriorityList.size + nodeLocalPriorityList.size
}
val numRackLocalOffers = {
val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers)
offersPriorityList ++= rackLocalPriorityList
rackLocalPriorityList.size
}
offersPriorityList ++= ClusterScheduler.prioritizeContainers(otherOffers)
var lastLoop = false
val lastLoopIndex = TASK_SCHEDULING_AGGRESSION match {
case TaskLocality.NODE_LOCAL => numNodeLocalOffers
case TaskLocality.RACK_LOCAL => numRackLocalOffers + numNodeLocalOffers
case TaskLocality.ANY => offersPriorityList.size
}
for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) {
do {
launchedTask = false
var loopCount = 0
for (i <- offersPriorityList) {
for (i <- 0 until offers.size) {
val execId = offers(i).executorId
val hostPort = offers(i).hostPort
// If last loop and within the lastLoopIndex, expand scope - else use null (which will use default/existing)
val overrideLocality = if (lastLoop && loopCount < lastLoopIndex) TASK_SCHEDULING_AGGRESSION else null
// If last loop, override waiting for host locality - we scheduled all local tasks already and there might be more available ...
loopCount += 1
manager.slaveOffer(execId, hostPort, availableCpus(i), overrideLocality) match {
case Some(task) =>
val host = offers(i).host
for (task <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) {
tasks(i) += task
val tid = task.taskId
taskIdToTaskSetId(tid) = manager.taskSet.id
taskSetTaskIds(manager.taskSet.id) += tid
taskIdToTaskSetId(tid) = taskSet.taskSet.id
taskSetTaskIds(taskSet.taskSet.id) += tid
taskIdToExecutorId(tid) = execId
activeExecutorIds += execId
executorsByHostPort(hostPort) += execId
executorsByHost(host) += execId
availableCpus(i) -= 1
launchedTask = true
case None => {}
}
}
// Loop once more - when lastLoop = true, then we try to schedule task on all nodes irrespective of
// data locality (we still go in order of priority : but that would not change anything since
// if data local tasks had been available, we would have scheduled them already)
if (lastLoop) {
// prevent more looping
launchedTask = false
} else if (!lastLoop && !launchedTask) {
// Do this only if TASK_SCHEDULING_AGGRESSION != NODE_LOCAL
if (TASK_SCHEDULING_AGGRESSION != TaskLocality.NODE_LOCAL) {
// fudge launchedTask to ensure we loop once more
launchedTask = true
// dont loop anymore
lastLoop = true
}
}
} while (launchedTask)
@ -401,7 +232,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
return tasks
}
}
def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
var taskSetToUpdate: Option[TaskSetManager] = None
@ -448,7 +278,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
backend.reviveOffers()
}
if (taskFailed) {
// Also revive offers if a task had failed for some reason other than host lost
backend.reviveOffers()
}
@ -503,7 +332,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
// Check for pending tasks in all our active jobs.
def hasPendingTasks(): Boolean = {
def hasPendingTasks: Boolean = {
synchronized {
rootPool.hasPendingTasks()
}
@ -514,7 +343,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
synchronized {
if (activeExecutorIds.contains(executorId)) {
val hostPort = executorIdToHostPort(executorId)
val hostPort = executorIdToHost(executorId)
logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason))
removeExecutor(executorId)
failedExecutor = Some(executorId)
@ -536,88 +365,63 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
/** Remove an executor from all our data structures and mark it as lost */
private def removeExecutor(executorId: String) {
activeExecutorIds -= executorId
val hostPort = executorIdToHostPort(executorId)
if (hostPortsAlive.contains(hostPort)) {
// DEBUG Code
Utils.checkHostPort(hostPort)
hostPortsAlive -= hostPort
hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(hostPort)._1, new HashSet[String]).remove(hostPort)
}
val execs = executorsByHostPort.getOrElse(hostPort, new HashSet)
val host = executorIdToHost(executorId)
val execs = executorsByHost.getOrElse(host, new HashSet)
execs -= executorId
if (execs.isEmpty) {
executorsByHostPort -= hostPort
executorsByHost -= host
}
executorIdToHostPort -= executorId
rootPool.executorLost(executorId, hostPort)
executorIdToHost -= executorId
rootPool.executorLost(executorId, host)
}
def executorGained(execId: String, hostPort: String) {
listener.executorGained(execId, hostPort)
def executorGained(execId: String, host: String) {
listener.executorGained(execId, host)
}
def getExecutorsAliveOnHost(host: String): Option[Set[String]] = {
Utils.checkHost(host)
val retval = hostToAliveHostPorts.get(host)
if (retval.isDefined) {
return Some(retval.get.toSet)
def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized {
executorsByHost.get(host).map(_.toSet)
}
None
def hasExecutorsAliveOnHost(host: String): Boolean = synchronized {
executorsByHost.contains(host)
}
def isExecutorAliveOnHostPort(hostPort: String): Boolean = {
// Even if hostPort is a host, it does not matter - it is just a specific check.
// But we do have to ensure that only hostPort get into hostPortsAlive !
// So no check against Utils.checkHostPort
hostPortsAlive.contains(hostPort)
def isExecutorAlive(execId: String): Boolean = synchronized {
activeExecutorIds.contains(execId)
}
// By default, rack is unknown
def getRackForHost(value: String): Option[String] = None
// By default, (cached) hosts for rack is unknown
def getCachedHostsForRack(rack: String): Option[Set[String]] = None
}
object ClusterScheduler {
// Used to 'spray' available containers across the available set to ensure too many containers on same host
// are not used up. Used in yarn mode and in task scheduling (when there are multiple containers available
// to execute a task)
// For example: yarn can returns more containers than we would have requested under ANY, this method
// prioritizes how to use the allocated containers.
// flatten the map such that the array buffer entries are spread out across the returned value.
// given <host, list[container]> == <h1, [c1 .. c5]>, <h2, [c1 .. c3]>, <h3, [c1, c2]>, <h4, c1>, <h5, c1>, i
// the return value would be something like : h1c1, h2c1, h3c1, h4c1, h5c1, h1c2, h2c2, h3c2, h1c3, h2c3, h1c4, h1c5
// We then 'use' the containers in this order (consuming only the top K from this list where
// K = number to be user). This is to ensure that if we have multiple eligible allocations,
// they dont end up allocating all containers on a small number of hosts - increasing probability of
// multiple container failure when a host goes down.
// Note, there is bias for keys with higher number of entries in value to be picked first (by design)
// Also note that invocation of this method is expected to have containers of same 'type'
// (host-local, rack-local, off-rack) and not across types : so that reordering is simply better from
// the available list - everything else being same.
// That is, we we first consume data local, then rack local and finally off rack nodes. So the
// prioritization from this method applies to within each category
/**
* Used to balance containers across hosts.
*
* Accepts a map of hosts to resource offers for that host, and returns a prioritized list of
* resource offers representing the order in which the offers should be used. The resource
* offers are ordered such that we'll allocate one container on each host before allocating a
* second container on any host, and so on, in order to reduce the damage if a host fails.
*
* For example, given <h1, [o1, o2, o3]>, <h2, [o4]>, <h1, [o5, o6]>, returns
* [o1, o5, o4, 02, o6, o3]
*/
def prioritizeContainers[K, T] (map: HashMap[K, ArrayBuffer[T]]): List[T] = {
val _keyList = new ArrayBuffer[K](map.size)
_keyList ++= map.keys
// order keyList based on population of value in map
val keyList = _keyList.sortWith(
(left, right) => map.get(left).getOrElse(Set()).size > map.get(right).getOrElse(Set()).size
(left, right) => map(left).size > map(right).size
)
val retval = new ArrayBuffer[T](keyList.size * 2)
var index = 0
var found = true
while (found){
while (found) {
found = false
for (key <- keyList) {
val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null)

View file

@ -29,60 +29,33 @@ import scala.math.min
import spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils}
import spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure}
import spark.TaskState.TaskState
import spark.scheduler.{ShuffleMapTask, Task, TaskResult, TaskSet}
import spark.scheduler._
import scala.Some
import spark.FetchFailed
import spark.ExceptionFailure
import spark.TaskResultTooBigFailure
import spark.util.{SystemClock, Clock}
private[spark] object TaskLocality
extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging {
// process local is expected to be used ONLY within tasksetmanager for now.
val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value
type TaskLocality = Value
def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = {
// Must not be the constraint.
assert (constraint != TaskLocality.PROCESS_LOCAL)
constraint match {
case TaskLocality.NODE_LOCAL =>
condition == TaskLocality.NODE_LOCAL
case TaskLocality.RACK_LOCAL =>
condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL
// For anything else, allow
case _ => true
}
}
def parse(str: String): TaskLocality = {
// better way to do this ?
try {
val retval = TaskLocality.withName(str)
// Must not specify PROCESS_LOCAL !
assert (retval != TaskLocality.PROCESS_LOCAL)
retval
} catch {
case nEx: NoSuchElementException => {
logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL")
// default to preserve earlier behavior
NODE_LOCAL
}
}
}
}
/**
* Schedules the tasks within a single TaskSet in the ClusterScheduler.
* Schedules the tasks within a single TaskSet in the ClusterScheduler. This class keeps track of
* the status of each task, retries tasks if they fail (up to a limited number of times), and
* handles locality-aware scheduling for this TaskSet via delay scheduling. The main interfaces
* to it are resourceOffer, which asks the TaskSet whether it wants to run a task on one node,
* and statusUpdate, which tells it that one of its tasks changed state (e.g. finished).
*
* THREADING: This class is designed to only be called from code with a lock on the
* ClusterScheduler (e.g. its event handlers). It should not be called from other threads.
*/
private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet)
extends TaskSetManager with Logging {
// Maximum time to wait to run a task in a preferred location (in ms)
val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong
private[spark] class ClusterTaskSetManager(
sched: ClusterScheduler,
val taskSet: TaskSet,
clock: Clock = SystemClock)
extends TaskSetManager
with Logging
{
// CPUs to request per task
val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toDouble
val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toInt
// Maximum times a task is allowed to fail before failing the job
val MAX_TASK_FAILURES = System.getProperty("spark.task.maxFailures", "4").toInt
@ -110,31 +83,27 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
var stageId = taskSet.stageId
var name = "TaskSet_"+taskSet.stageId.toString
var parent: Schedulable = null
// Last time when we launched a preferred task (for delay scheduling)
var lastPreferredLaunchTime = System.currentTimeMillis
// List of pending tasks for each node (process local to container).
// These collections are actually
// Set of pending tasks for each executor. These collections are actually
// treated as stacks, in which new tasks are added to the end of the
// ArrayBuffer and removed from the end. This makes it faster to detect
// tasks that repeatedly fail because whenever a task failed, it is put
// back at the head of the stack. They are also only cleaned up lazily;
// when a task is launched, it remains in all the pending lists except
// the one that it was launched from, but gets removed from them later.
private val pendingTasksForHostPort = new HashMap[String, ArrayBuffer[Int]]
private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]]
// List of pending tasks for each node.
// Essentially, similar to pendingTasksForHostPort, except at host level
// Set of pending tasks for each host. Similar to pendingTasksForExecutor,
// but at host level.
private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]]
// List of pending tasks for each node based on rack locality.
// Essentially, similar to pendingTasksForHost, except at rack level
private val pendingRackLocalTasksForHost = new HashMap[String, ArrayBuffer[Int]]
// Set of pending tasks for each rack -- similar to the above.
private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]]
// List containing pending tasks with no locality preferences
// Set containing pending tasks with no locality preferences.
val pendingTasksWithNoPrefs = new ArrayBuffer[Int]
// List containing all pending tasks (also used as a stack, as above)
// Set containing all pending tasks (also used as a stack, as above).
val allPendingTasks = new ArrayBuffer[Int]
// Tasks that can be speculated. Since these will be a small fraction of total
@ -144,25 +113,24 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
// Task index, start and finish time for each task attempt (indexed by task ID)
val taskInfos = new HashMap[Long, TaskInfo]
// Did the job fail?
// Did the TaskSet fail?
var failed = false
var causeOfFailure = ""
// How frequently to reprint duplicate exceptions in full, in milliseconds
val EXCEPTION_PRINT_INTERVAL =
System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong
// Map of recent exceptions (identified by string representation and
// top stack frame) to duplicate count (how many times the same
// exception has appeared) and time the full exception was
// printed. This should ideally be an LRU map that can drop old
// exceptions automatically.
// Map of recent exceptions (identified by string representation and top stack frame) to
// duplicate count (how many times the same exception has appeared) and time the full exception
// was printed. This should ideally be an LRU map that can drop old exceptions automatically.
val recentExceptions = HashMap[String, (Int, Long)]()
// Figure out the current map output tracker generation and set it on all tasks
val generation = sched.mapOutputTracker.getGeneration
logDebug("Generation for " + taskSet.id + ": " + generation)
// Figure out the current map output tracker epoch and set it on all tasks
val epoch = sched.mapOutputTracker.getEpoch
logDebug("Epoch for " + taskSet + ": " + epoch)
for (t <- tasks) {
t.generation = generation
t.epoch = epoch
}
// Add all our tasks to the pending lists. We do this in reverse order
@ -171,166 +139,86 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
addPendingTask(i)
}
// Note that it follows the hierarchy.
// if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and
// if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL
private def findPreferredLocations(
_taskPreferredLocations: Seq[String],
scheduler: ClusterScheduler,
taskLocality: TaskLocality.TaskLocality): HashSet[String] =
{
if (TaskLocality.PROCESS_LOCAL == taskLocality) {
// straight forward comparison ! Special case it.
val retval = new HashSet[String]()
scheduler.synchronized {
for (location <- _taskPreferredLocations) {
if (scheduler.isExecutorAliveOnHostPort(location)) {
retval += location
}
}
}
// Figure out which locality levels we have in our TaskSet, so we can do delay scheduling
val myLocalityLevels = computeValidLocalityLevels()
val localityWaits = myLocalityLevels.map(getLocalityWait) // Time to wait at each level
return retval
}
// Delay scheduling variables: we keep track of our current locality level and the time we
// last launched a task at that level, and move up a level when localityWaits[curLevel] expires.
// We then move down if we manage to launch a "more local" task.
var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels
var lastLaunchTime = clock.getTime() // Time we last launched a task at this level
val taskPreferredLocations = {
if (TaskLocality.NODE_LOCAL == taskLocality) {
_taskPreferredLocations
} else {
assert (TaskLocality.RACK_LOCAL == taskLocality)
// Expand set to include all 'seen' rack local hosts.
// This works since container allocation/management happens within master -
// so any rack locality information is updated in msater.
// Best case effort, and maybe sort of kludge for now ... rework it later ?
val hosts = new HashSet[String]
_taskPreferredLocations.foreach(h => {
val rackOpt = scheduler.getRackForHost(h)
if (rackOpt.isDefined) {
val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get)
if (hostsOpt.isDefined) {
hosts ++= hostsOpt.get
}
}
// Ensure that irrespective of what scheduler says, host is always added !
hosts += h
})
hosts
}
}
val retval = new HashSet[String]
scheduler.synchronized {
for (prefLocation <- taskPreferredLocations) {
val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(Utils.parseHostPort(prefLocation)._1)
if (aliveLocationsOpt.isDefined) {
retval ++= aliveLocationsOpt.get
}
}
}
retval
}
// Add a task to all the pending-task lists that it should be on.
private def addPendingTask(index: Int) {
// We can infer hostLocalLocations from rackLocalLocations by joining it against
// tasks(index).preferredLocations (with appropriate hostPort <-> host conversion).
// But not doing it for simplicity sake. If this becomes a performance issue, modify it.
val locs = tasks(index).preferredLocations
val processLocalLocations = findPreferredLocations(locs, sched, TaskLocality.PROCESS_LOCAL)
val hostLocalLocations = findPreferredLocations(locs, sched, TaskLocality.NODE_LOCAL)
val rackLocalLocations = findPreferredLocations(locs, sched, TaskLocality.RACK_LOCAL)
if (rackLocalLocations.size == 0) {
// Current impl ensures this.
assert (processLocalLocations.size == 0)
assert (hostLocalLocations.size == 0)
pendingTasksWithNoPrefs += index
} else {
// process local locality
for (hostPort <- processLocalLocations) {
// DEBUG Code
Utils.checkHostPort(hostPort)
val hostPortList = pendingTasksForHostPort.getOrElseUpdate(hostPort, ArrayBuffer())
hostPortList += index
}
// host locality (includes process local)
for (hostPort <- hostLocalLocations) {
// DEBUG Code
Utils.checkHostPort(hostPort)
val host = Utils.parseHostPort(hostPort)._1
val hostList = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer())
hostList += index
}
// rack locality (includes process local and host local)
for (rackLocalHostPort <- rackLocalLocations) {
// DEBUG Code
Utils.checkHostPort(rackLocalHostPort)
val rackLocalHost = Utils.parseHostPort(rackLocalHostPort)._1
val list = pendingRackLocalTasksForHost.getOrElseUpdate(rackLocalHost, ArrayBuffer())
/**
* Add a task to all the pending-task lists that it should be on. If readding is set, we are
* re-adding the task so only include it in each list if it's not already there.
*/
private def addPendingTask(index: Int, readding: Boolean = false) {
// Utility method that adds `index` to a list only if readding=false or it's not already there
def addTo(list: ArrayBuffer[Int]) {
if (!readding || !list.contains(index)) {
list += index
}
}
allPendingTasks += index
var hadAliveLocations = false
for (loc <- tasks(index).preferredLocations) {
for (execId <- loc.executorId) {
if (sched.isExecutorAlive(execId)) {
addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer))
hadAliveLocations = true
}
}
if (sched.hasExecutorsAliveOnHost(loc.host)) {
addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer))
for (rack <- sched.getRackForHost(loc.host)) {
addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer))
}
hadAliveLocations = true
}
}
// Return the pending tasks list for a given host port (process local), or an empty list if
// there is no map entry for that host
private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = {
// DEBUG Code
Utils.checkHostPort(hostPort)
pendingTasksForHostPort.getOrElse(hostPort, ArrayBuffer())
if (!hadAliveLocations) {
// Even though the task might've had preferred locations, all of those hosts or executors
// are dead; put it in the no-prefs list so we can schedule it elsewhere right away.
addTo(pendingTasksWithNoPrefs)
}
// Return the pending tasks list for a given host, or an empty list if
// there is no map entry for that host
private def getPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = {
val host = Utils.parseHostPort(hostPort)._1
if (!readding) {
allPendingTasks += index // No point scanning this whole list to find the old task there
}
}
/**
* Return the pending tasks list for a given executor ID, or an empty list if
* there is no map entry for that host
*/
private def getPendingTasksForExecutor(executorId: String): ArrayBuffer[Int] = {
pendingTasksForExecutor.getOrElse(executorId, ArrayBuffer())
}
/**
* Return the pending tasks list for a given host, or an empty list if
* there is no map entry for that host
*/
private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = {
pendingTasksForHost.getOrElse(host, ArrayBuffer())
}
// Return the pending tasks (rack level) list for a given host, or an empty list if
// there is no map entry for that host
private def getRackLocalPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = {
val host = Utils.parseHostPort(hostPort)._1
pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer())
/**
* Return the pending rack-local task list for a given rack, or an empty list if
* there is no map entry for that rack
*/
private def getPendingTasksForRack(rack: String): ArrayBuffer[Int] = {
pendingTasksForRack.getOrElse(rack, ArrayBuffer())
}
// Number of pending tasks for a given host Port (which would be process local)
override def numPendingTasksForHostPort(hostPort: String): Int = {
getPendingTasksForHostPort(hostPort).count { index =>
copiesRunning(index) == 0 && !finished(index)
}
}
// Number of pending tasks for a given host (which would be data local)
override def numPendingTasksForHost(hostPort: String): Int = {
getPendingTasksForHost(hostPort).count { index =>
copiesRunning(index) == 0 && !finished(index)
}
}
// Number of pending rack local tasks for a given host
override def numRackLocalPendingTasksForHost(hostPort: String): Int = {
getRackLocalPendingTasksForHost(hostPort).count { index =>
copiesRunning(index) == 0 && !finished(index)
}
}
// Dequeue a pending task from the given list and return its index.
// Return None if the list is empty.
// This method also cleans up any tasks in the list that have already
// been launched, since we want that to happen lazily.
/**
* Dequeue a pending task from the given list and return its index.
* Return None if the list is empty.
* This method also cleans up any tasks in the list that have already
* been launched, since we want that to happen lazily.
*/
private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = {
while (!list.isEmpty) {
val index = list.last
@ -342,191 +230,158 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
return None
}
// Return a speculative task for a given host if any are available. The task should not have an
// attempt running on this host, in case the host is slow. In addition, if locality is set, the
// task must have a preference for this host/rack/no preferred locations at all.
private def findSpeculativeTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = {
/** Check whether a task is currently running an attempt on a given host */
private def hasAttemptOnHost(taskIndex: Int, host: String): Boolean = {
!taskAttempts(taskIndex).exists(_.host == host)
}
assert (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL))
/**
* Return a speculative task for a given executor if any are available. The task should not have
* an attempt running on this host, in case the host is slow. In addition, the task should meet
* the given locality constraint.
*/
private def findSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value)
: Option[(Int, TaskLocality.Value)] =
{
speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set
if (speculatableTasks.size > 0) {
val localTask = speculatableTasks.find { index =>
val locations = findPreferredLocations(tasks(index).preferredLocations, sched,
TaskLocality.NODE_LOCAL)
val attemptLocs = taskAttempts(index).map(_.hostPort)
(locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort)
if (!speculatableTasks.isEmpty) {
// Check for process-local or preference-less tasks; note that tasks can be process-local
// on multiple nodes when we replicate cached blocks, as in Spark Streaming
for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
val prefs = tasks(index).preferredLocations
val executors = prefs.flatMap(_.executorId)
if (prefs.size == 0 || executors.contains(execId)) {
speculatableTasks -= index
return Some((index, TaskLocality.PROCESS_LOCAL))
}
}
if (localTask != None) {
speculatableTasks -= localTask.get
return localTask
// Check for node-local tasks
if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) {
for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
val locations = tasks(index).preferredLocations.map(_.host)
if (locations.contains(host)) {
speculatableTasks -= index
return Some((index, TaskLocality.NODE_LOCAL))
}
}
}
// check for rack locality
// Check for rack-local tasks
if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) {
val rackTask = speculatableTasks.find { index =>
val locations = findPreferredLocations(tasks(index).preferredLocations, sched,
TaskLocality.RACK_LOCAL)
val attemptLocs = taskAttempts(index).map(_.hostPort)
locations.contains(hostPort) && !attemptLocs.contains(hostPort)
for (rack <- sched.getRackForHost(host)) {
for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost)
if (racks.contains(rack)) {
speculatableTasks -= index
return Some((index, TaskLocality.RACK_LOCAL))
}
}
if (rackTask != None) {
speculatableTasks -= rackTask.get
return rackTask
}
}
// Any task ...
// Check for non-local tasks
if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) {
// Check for attemptLocs also ?
val nonLocalTask = speculatableTasks.find { i =>
!taskAttempts(i).map(_.hostPort).contains(hostPort)
}
if (nonLocalTask != None) {
speculatableTasks -= nonLocalTask.get
return nonLocalTask
for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
speculatableTasks -= index
return Some((index, TaskLocality.ANY))
}
}
}
return None
}
// Dequeue a pending task for a given node and return its index.
// If localOnly is set to false, allow non-local tasks as well.
private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = {
val processLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort))
if (processLocalTask != None) {
return processLocalTask
/**
* Dequeue a pending task for a given node and return its index and locality level.
* Only search for tasks matching the given locality constraint.
*/
private def findTask(execId: String, host: String, locality: TaskLocality.Value)
: Option[(Int, TaskLocality.Value)] =
{
for (index <- findTaskFromList(getPendingTasksForExecutor(execId))) {
return Some((index, TaskLocality.PROCESS_LOCAL))
}
val localTask = findTaskFromList(getPendingTasksForHost(hostPort))
if (localTask != None) {
return localTask
if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) {
for (index <- findTaskFromList(getPendingTasksForHost(host))) {
return Some((index, TaskLocality.NODE_LOCAL))
}
}
if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) {
val rackLocalTask = findTaskFromList(getRackLocalPendingTasksForHost(hostPort))
if (rackLocalTask != None) {
return rackLocalTask
for {
rack <- sched.getRackForHost(host)
index <- findTaskFromList(getPendingTasksForRack(rack))
} {
return Some((index, TaskLocality.RACK_LOCAL))
}
}
// Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to
// failed tasks later rather than sooner.
// TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down).
val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs)
if (noPrefTask != None) {
return noPrefTask
// Look for no-pref tasks after rack-local tasks since they can run anywhere.
for (index <- findTaskFromList(pendingTasksWithNoPrefs)) {
return Some((index, TaskLocality.PROCESS_LOCAL))
}
if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) {
val nonLocalTask = findTaskFromList(allPendingTasks)
if (nonLocalTask != None) {
return nonLocalTask
for (index <- findTaskFromList(allPendingTasks)) {
return Some((index, TaskLocality.ANY))
}
}
// Finally, if all else has failed, find a speculative task
return findSpeculativeTask(hostPort, locality)
return findSpeculativeTask(execId, host, locality)
}
private def isProcessLocalLocation(task: Task[_], hostPort: String): Boolean = {
Utils.checkHostPort(hostPort)
val locs = task.preferredLocations
locs.contains(hostPort)
}
private def isHostLocalLocation(task: Task[_], hostPort: String): Boolean = {
val locs = task.preferredLocations
// If no preference, consider it as host local
if (locs.isEmpty) return true
val host = Utils.parseHostPort(hostPort)._1
locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined
}
// Does a host count as a rack local preferred location for a task?
// (assumes host is NOT preferred location).
// This is true if either the task has preferred locations and this host is one, or it has
// no preferred locations (in which we still count the launch as preferred).
private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = {
val locs = task.preferredLocations
val preferredRacks = new HashSet[String]()
for (preferredHost <- locs) {
val rack = sched.getRackForHost(preferredHost)
if (None != rack) preferredRacks += rack.get
}
if (preferredRacks.isEmpty) return false
val hostRack = sched.getRackForHost(hostPort)
return None != hostRack && preferredRacks.contains(hostRack.get)
}
// Respond to an offer of a single slave from the scheduler by finding a task
override def slaveOffer(
/**
* Respond to an offer of a single slave from the scheduler by finding a task
*/
override def resourceOffer(
execId: String,
hostPort: String,
availableCpus: Double,
overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] =
host: String,
availableCpus: Int,
maxLocality: TaskLocality.TaskLocality)
: Option[TaskDescription] =
{
if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) {
// If explicitly specified, use that
val locality = if (overrideLocality != null) overrideLocality else {
// expand only if we have waited for more than LOCALITY_WAIT for a host local task ...
val time = System.currentTimeMillis
if (time - lastPreferredLaunchTime < LOCALITY_WAIT) {
TaskLocality.NODE_LOCAL
} else {
TaskLocality.ANY
}
val curTime = clock.getTime()
var allowedLocality = getAllowedLocalityLevel(curTime)
if (allowedLocality > maxLocality) {
allowedLocality = maxLocality // We're not allowed to search for farther-away tasks
}
findTask(hostPort, locality) match {
case Some(index) => {
// Found a task; do some bookkeeping and return a Mesos task for it
findTask(execId, host, allowedLocality) match {
case Some((index, taskLocality)) => {
// Found a task; do some bookkeeping and return a task description
val task = tasks(index)
val taskId = sched.newTaskId()
// Figure out whether this should count as a preferred launch
val taskLocality =
if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL
else if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL
else if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL
else TaskLocality.ANY
val prefStr = taskLocality.toString
logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format(
taskSet.id, index, taskId, execId, hostPort, prefStr))
taskSet.id, index, taskId, execId, host, taskLocality))
// Do various bookkeeping
copiesRunning(index) += 1
val time = System.currentTimeMillis
val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality)
val info = new TaskInfo(taskId, index, curTime, execId, host, taskLocality)
taskInfos(taskId) = info
taskAttempts(index) = info :: taskAttempts(index)
if (taskLocality == TaskLocality.PROCESS_LOCAL || taskLocality == TaskLocality.NODE_LOCAL) {
lastPreferredLaunchTime = time
}
// Update our locality level for delay scheduling
currentLocalityIndex = getLocalityIndex(taskLocality)
lastLaunchTime = curTime
// Serialize and return the task
val startTime = System.currentTimeMillis
val startTime = clock.getTime()
// We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here
// we assume the task can be serialized without exceptions.
val serializedTask = Task.serializeWithDependencies(
task, sched.sc.addedFiles, sched.sc.addedJars, ser)
val timeTaken = System.currentTimeMillis - startTime
val timeTaken = clock.getTime() - startTime
increaseRunningTasks(1)
logInfo("Serialized task %s:%d as %d bytes in %d ms".format(
taskSet.id, index, serializedTask.limit, timeTaken))
val taskName = "task %s:%d".format(taskSet.id, index)
if (taskAttempts(index).size == 1)
taskStarted(task,info)
return Some(new TaskDescription(taskId, execId, taskName, serializedTask))
return Some(new TaskDescription(taskId, execId, taskName, index, serializedTask))
}
case _ =>
}
@ -534,6 +389,35 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
return None
}
/**
* Get the level we can launch tasks according to delay scheduling, based on current wait time.
*/
private def getAllowedLocalityLevel(curTime: Long): TaskLocality.TaskLocality = {
while (curTime - lastLaunchTime >= localityWaits(currentLocalityIndex) &&
currentLocalityIndex < myLocalityLevels.length - 1)
{
// Jump to the next locality level, and remove our waiting time for the current one since
// we don't want to count it again on the next one
lastLaunchTime += localityWaits(currentLocalityIndex)
currentLocalityIndex += 1
}
myLocalityLevels(currentLocalityIndex)
}
/**
* Find the index in myLocalityLevels for a given locality. This is also designed to work with
* localities that are not in myLocalityLevels (in case we somehow get those) by returning the
* next-biggest level we have. Uses the fact that the last value in myLocalityLevels is ANY.
*/
def getLocalityIndex(locality: TaskLocality.TaskLocality): Int = {
var index = 0
while (locality > myLocalityLevels(index)) {
index += 1
}
index
}
/** Called by cluster scheduler when one of our tasks changes state */
override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
SparkEnv.set(env)
state match {
@ -566,7 +450,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
if (!finished(index)) {
tasksFinished += 1
logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format(
tid, info.duration, info.hostPort, tasksFinished, numTasks))
tid, info.duration, info.host, tasksFinished, numTasks))
// Deserialize task result and pass it to the scheduler
try {
val result = ser.deserialize[TaskResult[_]](serializedData)
@ -626,7 +510,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
case ef: ExceptionFailure =>
sched.listener.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null))
val key = ef.description
val now = System.currentTimeMillis
val now = clock.getTime()
val (printFull, dupCount) = {
if (recentExceptions.contains(key)) {
val (dupCount, printTime) = recentExceptions(key)
@ -698,44 +582,33 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
}
}
// TODO(xiajunluan): for now we just find Pool not TaskSetManager
// we can extend this function in future if needed
override def getSchedulableByName(name: String): Schedulable = {
return null
}
override def addSchedulable(schedulable:Schedulable) {
// nothing
}
override def addSchedulable(schedulable: Schedulable) {}
override def removeSchedulable(schedulable:Schedulable) {
// nothing
}
override def removeSchedulable(schedulable: Schedulable) {}
override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = {
var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager]
var sortedTaskSetQueue = ArrayBuffer[TaskSetManager](this)
sortedTaskSetQueue += this
return sortedTaskSetQueue
}
override def executorLost(execId: String, hostPort: String) {
/** Called by cluster scheduler when an executor is lost so we can re-enqueue our tasks */
override def executorLost(execId: String, host: String) {
logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id)
// If some task has preferred locations only on hostname, and there are no more executors there,
// put it in the no-prefs list to avoid the wait from delay scheduling
// host local tasks - should we push this to rack local or no pref list ? For now, preserving
// behavior and moving to no prefs list. Note, this was done due to impliations related to
// 'waiting' for data local tasks, etc.
// Note: NOT checking process local list - since host local list is super set of that. We need
// to ad to no prefs only if there is no host local node for the task (not if there is no
// process local node for the task)
for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) {
val newLocs = findPreferredLocations(
tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL)
if (newLocs.isEmpty) {
pendingTasksWithNoPrefs += index
// Re-enqueue pending tasks for this host based on the status of the cluster -- for example, a
// task that used to have locations on only this host might now go to the no-prefs list. Note
// that it's okay if we add a task to the same queue twice (if it had multiple preferred
// locations), because findTaskFromList will skip already-running tasks.
for (index <- getPendingTasksForExecutor(execId)) {
addPendingTask(index, readding=true)
}
for (index <- getPendingTasksForHost(host)) {
addPendingTask(index, readding=true)
}
// Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage
@ -775,7 +648,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt
logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation)
if (tasksFinished >= minFinishedForSpeculation) {
val time = System.currentTimeMillis()
val time = clock.getTime()
val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray
Arrays.sort(durations)
val medianDuration = durations(min((0.5 * numTasks).round.toInt, durations.size - 1))
@ -789,7 +662,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
!speculatableTasks.contains(index)) {
logInfo(
"Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format(
taskSet.id, index, info.hostPort, threshold))
taskSet.id, index, info.host, threshold))
speculatableTasks += index
foundTasks = true
}
@ -801,4 +674,39 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet:
override def hasPendingTasks(): Boolean = {
numTasks > 0 && tasksFinished < numTasks
}
private def getLocalityWait(level: TaskLocality.TaskLocality): Long = {
val defaultWait = System.getProperty("spark.locality.wait", "3000")
level match {
case TaskLocality.PROCESS_LOCAL =>
System.getProperty("spark.locality.wait.process", defaultWait).toLong
case TaskLocality.NODE_LOCAL =>
System.getProperty("spark.locality.wait.node", defaultWait).toLong
case TaskLocality.RACK_LOCAL =>
System.getProperty("spark.locality.wait.rack", defaultWait).toLong
case TaskLocality.ANY =>
0L
}
}
/**
* Compute the locality levels used in this TaskSet. Assumes that all tasks have already been
* added to queues using addPendingTask.
*/
private def computeValidLocalityLevels(): Array[TaskLocality.TaskLocality] = {
import TaskLocality.{PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY}
val levels = new ArrayBuffer[TaskLocality.TaskLocality]
if (!pendingTasksForExecutor.isEmpty && getLocalityWait(PROCESS_LOCAL) != 0) {
levels += PROCESS_LOCAL
}
if (!pendingTasksForHost.isEmpty && getLocalityWait(NODE_LOCAL) != 0) {
levels += NODE_LOCAL
}
if (!pendingTasksForRack.isEmpty && getLocalityWait(RACK_LOCAL) != 0) {
levels += RACK_LOCAL
}
levels += ANY
logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", "))
levels.toArray
}
}

View file

@ -26,6 +26,7 @@ import akka.dispatch.Await
import akka.pattern.ask
import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent}
import akka.util.Duration
import akka.util.duration._
import spark.{Utils, SparkException, Logging, TaskState}
import spark.scheduler.cluster.StandaloneClusterMessages._
@ -37,15 +38,15 @@ import spark.scheduler.cluster.StandaloneClusterMessages._
*/
private[spark]
class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem)
extends SchedulerBackend with Logging {
extends SchedulerBackend with Logging
{
// Use an atomic variable to track total number of cores in the cluster for simplicity and speed
var totalCoreCount = new AtomicInteger(0)
class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor {
private val executorActor = new HashMap[String, ActorRef]
private val executorAddress = new HashMap[String, Address]
private val executorHostPort = new HashMap[String, String]
private val executorHost = new HashMap[String, String]
private val freeCores = new HashMap[String, Int]
private val actorToExecutorId = new HashMap[ActorRef, String]
private val addressToExecutorId = new HashMap[Address, String]
@ -53,6 +54,10 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
override def preStart() {
// Listen for remote client disconnection events, since they don't go through Akka's watch()
context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
// Periodically revive offers to allow delay scheduling to work
val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong
context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers)
}
def receive = {
@ -65,7 +70,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
sender ! RegisteredExecutor(sparkProperties)
context.watch(sender)
executorActor(executorId) = sender
executorHostPort(executorId) = hostPort
executorHost(executorId) = Utils.parseHostPort(hostPort)._1
freeCores(executorId) = cores
executorAddress(executorId) = sender.path.address
actorToExecutorId(sender) = executorId
@ -105,13 +110,13 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
// Make fake resource offers on all executors
def makeOffers() {
launchTasks(scheduler.resourceOffers(
executorHostPort.toArray.map {case (id, hostPort) => new WorkerOffer(id, hostPort, freeCores(id))}))
executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))}))
}
// Make fake resource offers on just one executor
def makeOffers(executorId: String) {
launchTasks(scheduler.resourceOffers(
Seq(new WorkerOffer(executorId, executorHostPort(executorId), freeCores(executorId)))))
Seq(new WorkerOffer(executorId, executorHost(executorId), freeCores(executorId)))))
}
// Launch tasks returned by a set of resource offers
@ -130,9 +135,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
actorToExecutorId -= executorActor(executorId)
addressToExecutorId -= executorAddress(executorId)
executorActor -= executorId
executorHostPort -= executorId
executorHost -= executorId
freeCores -= executorId
executorHostPort -= executorId
totalCoreCount.addAndGet(-numCores)
scheduler.executorLost(executorId, SlaveLost(reason))
}

View file

@ -24,6 +24,7 @@ private[spark] class TaskDescription(
val taskId: Long,
val executorId: String,
val name: String,
val index: Int, // Index within this task's TaskSet
_serializedTask: ByteBuffer)
extends Serializable {
@ -31,4 +32,6 @@ private[spark] class TaskDescription(
private val buffer = new SerializableBuffer(_serializedTask)
def serializedTask: ByteBuffer = buffer.value
override def toString: String = "TaskDescription(TID=%d, index=%d)".format(taskId, index)
}

View file

@ -28,11 +28,9 @@ class TaskInfo(
val index: Int,
val launchTime: Long,
val executorId: String,
val hostPort: String,
val host: String,
val taskLocality: TaskLocality.TaskLocality) {
Utils.checkHostPort(hostPort, "Expected hostport")
var finishTime: Long = 0
var failed = false

View file

@ -15,13 +15,18 @@
* limitations under the License.
*/
package org.apache.hadoop.mapred
package spark.scheduler.cluster
trait HadoopMapRedUtil {
def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContext(conf, jobId)
def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId)
private[spark] object TaskLocality
extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY")
{
// process local is expected to be used ONLY within tasksetmanager for now.
val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value
def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier,
jobId, isMap, taskId, attemptId)
type TaskLocality = Value
def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = {
condition <= constraint
}
}

View file

@ -22,6 +22,15 @@ import java.nio.ByteBuffer
import spark.TaskState.TaskState
import spark.scheduler.TaskSet
/**
* Tracks and schedules the tasks within a single TaskSet. This class keeps track of the status of
* each task and is responsible for retries on failure and locality. The main interfaces to it
* are resourceOffer, which asks the TaskSet whether it wants to run a task on one node, and
* statusUpdate, which tells it that one of its tasks changed state (e.g. finished).
*
* THREADING: This class is designed to only be called from code with a lock on the TaskScheduler
* (e.g. its event handlers). It should not be called from other threads.
*/
private[spark] trait TaskSetManager extends Schedulable {
def schedulableQueue = null
@ -29,17 +38,12 @@ private[spark] trait TaskSetManager extends Schedulable {
def taskSet: TaskSet
def slaveOffer(
def resourceOffer(
execId: String,
hostPort: String,
availableCpus: Double,
overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription]
def numPendingTasksForHostPort(hostPort: String): Int
def numRackLocalPendingTasksForHost(hostPort: String): Int
def numPendingTasksForHost(hostPort: String): Int
host: String,
availableCpus: Int,
maxLocality: TaskLocality.TaskLocality)
: Option[TaskDescription]
def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer)

View file

@ -21,5 +21,4 @@ package spark.scheduler.cluster
* Represents free resources available on an executor.
*/
private[spark]
class WorkerOffer(val executorId: String, val hostPort: String, val cores: Int) {
}
class WorkerOffer(val executorId: String, val host: String, val cores: Int)

View file

@ -141,7 +141,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc:
for (manager <- sortedTaskSetQueue) {
do {
launchTask = false
manager.slaveOffer(null, null, freeCpuCores) match {
manager.resourceOffer(null, null, freeCpuCores, null) match {
case Some(task) =>
tasks += task
taskIdToTaskSetId(task.taskId) = manager.taskSet.id

View file

@ -98,14 +98,15 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
return None
}
override def slaveOffer(
override def resourceOffer(
execId: String,
hostPort: String,
availableCpus: Double,
overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] =
host: String,
availableCpus: Int,
maxLocality: TaskLocality.TaskLocality)
: Option[TaskDescription] =
{
SparkEnv.set(sched.env)
logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format(
logDebug("availableCpus:%d, numFinished:%d, numTasks:%d".format(
availableCpus.toInt, numFinished, numTasks))
if (availableCpus > 0 && numFinished < numTasks) {
findTask() match {
@ -124,25 +125,13 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
copiesRunning(index) += 1
increaseRunningTasks(1)
taskStarted(task, info)
return Some(new TaskDescription(taskId, null, taskName, bytes))
return Some(new TaskDescription(taskId, null, taskName, index, bytes))
case None => {}
}
}
return None
}
override def numPendingTasksForHostPort(hostPort: String): Int = {
return 0
}
override def numRackLocalPendingTasksForHost(hostPort :String): Int = {
return 0
}
override def numPendingTasksForHost(hostPort: String): Int = {
return 0
}
override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
SparkEnv.set(env)
state match {

View file

@ -1004,43 +1004,43 @@ private[spark] object BlockManager extends Logging {
}
}
def blockIdsToExecutorLocations(blockIds: Array[String], env: SparkEnv, blockManagerMaster: BlockManagerMaster = null): HashMap[String, List[String]] = {
def blockIdsToBlockManagers(
blockIds: Array[String],
env: SparkEnv,
blockManagerMaster: BlockManagerMaster = null)
: Map[String, Seq[BlockManagerId]] =
{
// env == null and blockManagerMaster != null is used in tests
assert (env != null || blockManagerMaster != null)
val locationBlockIds: Seq[Seq[BlockManagerId]] =
if (env != null) {
val blockLocations: Seq[Seq[BlockManagerId]] = if (env != null) {
env.blockManager.getLocationBlockIds(blockIds)
} else {
blockManagerMaster.getLocations(blockIds)
}
// Convert from block master locations to executor locations (we need that for task scheduling)
val executorLocations = new HashMap[String, List[String]]()
val blockManagers = new HashMap[String, Seq[BlockManagerId]]
for (i <- 0 until blockIds.length) {
val blockId = blockIds(i)
val blockLocations = locationBlockIds(i)
val executors = new HashSet[String]()
if (env != null) {
for (bkLocation <- blockLocations) {
val executorHostPort = env.resolveExecutorIdToHostPort(bkLocation.executorId, bkLocation.host)
executors += executorHostPort
// logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort)
}
} else {
// Typically while testing, etc - revert to simply using host.
for (bkLocation <- blockLocations) {
executors += bkLocation.host
// logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort)
blockManagers(blockIds(i)) = blockLocations(i)
}
blockManagers.toMap
}
executorLocations.put(blockId, executors.toSeq.toList)
def blockIdsToExecutorIds(
blockIds: Array[String],
env: SparkEnv,
blockManagerMaster: BlockManagerMaster = null)
: Map[String, Seq[String]] =
{
blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId))
}
executorLocations
def blockIdsToHosts(
blockIds: Array[String],
env: SparkEnv,
blockManagerMaster: BlockManagerMaster = null)
: Map[String, Seq[String]] =
{
blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host))
}
}

View file

@ -21,9 +21,10 @@ import javax.servlet.http.{HttpServletResponse, HttpServletRequest}
import scala.annotation.tailrec
import scala.util.{Try, Success, Failure}
import scala.util.parsing.json.JSONType
import scala.xml.Node
import net.liftweb.json.{JValue, pretty, render}
import org.eclipse.jetty.server.{Server, Request, Handler}
import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler}
import org.eclipse.jetty.util.thread.QueuedThreadPool
@ -38,8 +39,8 @@ private[spark] object JettyUtils extends Logging {
type Responder[T] = HttpServletRequest => T
// Conversions from various types of Responder's to jetty Handlers
implicit def jsonResponderToHandler(responder: Responder[JSONType]): Handler =
createHandler(responder, "text/json", (in: JSONType) => in.toString)
implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler =
createHandler(responder, "text/json", (in: JValue) => pretty(render(in)))
implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler =
createHandler(responder, "text/html", (in: Seq[Node]) => "<!DOCTYPE html>" + in.toString)

View file

@ -30,7 +30,7 @@ import spark.ui.JettyUtils._
/** Top level user interface for Spark */
private[spark] class SparkUI(sc: SparkContext) extends Logging {
val host = Utils.localHostName()
val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName())
val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt
var boundPort: Option[Int] = None
var server: Option[Server] = None
@ -82,6 +82,6 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging {
}
private[spark] object SparkUI {
val DEFAULT_PORT = "33000"
val DEFAULT_PORT = "3030"
val STATIC_RESOURCE_DIR = "spark/ui/static"
}

View file

@ -156,7 +156,7 @@ private[spark] class StagePage(parent: JobProgressUI) {
<td>{info.taskId}</td>
<td>{info.status}</td>
<td>{info.taskLocality}</td>
<td>{info.hostPort}</td>
<td>{info.host}</td>
<td>{dateFmt.format(new Date(info.launchTime))}</td>
<td sorttable_customkey={duration.toString}>
{formatDuration}

View file

@ -0,0 +1,29 @@
/*
* 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 spark.util
/**
* An interface to represent clocks, so that they can be mocked out in unit tests.
*/
private[spark] trait Clock {
def getTime(): Long
}
private[spark] object SystemClock extends Clock {
def getTime(): Long = System.currentTimeMillis()
}

View file

@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package spark.util
/**
* A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to
* minimize object allocation.
*
* @param _1 Element 1 of this MutablePair
* @param _2 Element 2 of this MutablePair
*/
case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T1,
@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T2]
(var _1: T1, var _2: T2)
extends Product2[T1, T2]
{
override def toString = "(" + _1 + "," + _2 + ")"
override def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[_,_]]
}

View file

@ -99,7 +99,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
test("ShuffledRDD") {
testCheckpointing(rdd => {
// Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD
new ShuffledRDD(rdd.map(x => (x % 2, 1)), partitioner)
new ShuffledRDD[Int, Int, (Int, Int)](rdd.map(x => (x % 2, 1)), partitioner)
})
}

View file

@ -112,22 +112,22 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
"akka://spark@localhost:" + boundPort + "/user/MapOutputTracker")
masterTracker.registerShuffle(10, 1)
masterTracker.incrementGeneration()
slaveTracker.updateGeneration(masterTracker.getGeneration)
masterTracker.incrementEpoch()
slaveTracker.updateEpoch(masterTracker.getEpoch)
intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) }
val compressedSize1000 = MapOutputTracker.compressSize(1000L)
val size1000 = MapOutputTracker.decompressSize(compressedSize1000)
masterTracker.registerMapOutput(10, 0, new MapStatus(
BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000)))
masterTracker.incrementGeneration()
slaveTracker.updateGeneration(masterTracker.getGeneration)
masterTracker.incrementEpoch()
slaveTracker.updateEpoch(masterTracker.getEpoch)
assert(slaveTracker.getServerStatuses(10, 0).toSeq ===
Seq((BlockManagerId("a", "hostA", 1000, 0), size1000)))
masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0))
masterTracker.incrementGeneration()
slaveTracker.updateGeneration(masterTracker.getGeneration)
masterTracker.incrementEpoch()
slaveTracker.updateEpoch(masterTracker.getEpoch)
intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) }
// failure should be cached

View file

@ -21,16 +21,11 @@ import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashSet
import org.scalatest.FunSuite
import org.scalatest.prop.Checkers
import org.scalacheck.Arbitrary._
import org.scalacheck.Gen
import org.scalacheck.Prop._
import com.google.common.io.Files
import spark.rdd.ShuffledRDD
import spark.SparkContext._
class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext {
test("groupByKey") {
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)))

View file

@ -22,7 +22,8 @@ import org.scalatest.FunSuite
import org.scalatest.concurrent.Timeouts._
import org.scalatest.time.{Span, Millis}
import spark.SparkContext._
import spark.rdd.{CoalescedRDD, CoGroupedRDD, EmptyRDD, PartitionPruningRDD, ShuffledRDD}
import spark.rdd._
import scala.collection.parallel.mutable
class RDDSuite extends FunSuite with SharedSparkContext {
@ -170,9 +171,69 @@ class RDDSuite extends FunSuite with SharedSparkContext {
// we can optionally shuffle to keep the upstream parallel
val coalesced5 = data.coalesce(1, shuffle = true)
assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _]] !=
assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _, _]] !=
null)
}
test("cogrouped RDDs with locality") {
val data3 = sc.makeRDD(List((1,List("a","c")), (2,List("a","b","c")), (3,List("b"))))
val coal3 = data3.coalesce(3)
val list3 = coal3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition].preferredLocation)
assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped")
// RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5
val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i+2)).map{ j => "m" + (j%6)})))
val coalesced1 = data.coalesce(3)
assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing")
val splits = coalesced1.glom().collect().map(_.toList).toList
assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length)
assert(splits.forall(_.length >= 1) === true, "Some partitions were empty")
// If we try to coalesce into more partitions than the original RDD, it should just
// keep the original number of partitions.
val coalesced4 = data.coalesce(20)
val listOfLists = coalesced4.glom().collect().map(_.toList).toList
val sortedList = listOfLists.sortWith{ (x, y) => !x.isEmpty && (y.isEmpty || (x(0) < y(0))) }
assert( sortedList === (1 to 9).
map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back")
}
test("cogrouped RDDs with locality, large scale (10K partitions)") {
// large scale experiment
import collection.mutable
val rnd = scala.util.Random
val partitions = 10000
val numMachines = 50
val machines = mutable.ListBuffer[String]()
(1 to numMachines).foreach(machines += "m"+_)
val blocks = (1 to partitions).map(i =>
{ (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) } )
val data2 = sc.makeRDD(blocks)
val coalesced2 = data2.coalesce(numMachines*2)
// test that you get over 90% locality in each group
val minLocality = coalesced2.partitions
.map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction)
.foldLeft(1.)((perc, loc) => math.min(perc,loc))
assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%")
// test that the groups are load balanced with 100 +/- 20 elements in each
val maxImbalance = coalesced2.partitions
.map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size)
.foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev))
assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance)
val data3 = sc.makeRDD(blocks).map(i => i*2) // derived RDD to test *current* pref locs
val coalesced3 = data3.coalesce(numMachines*2)
val minLocality2 = coalesced3.partitions
.map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction)
.foldLeft(1.)((perc, loc) => math.min(perc,loc))
assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " +
(minLocality2*100.).toInt + "%")
}
test("zipped RDDs") {
val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)

View file

@ -20,8 +20,11 @@ package spark
import org.scalatest.FunSuite
import org.scalatest.matchers.ShouldMatchers
import spark.rdd.ShuffledRDD
import spark.SparkContext._
import spark.ShuffleSuite.NonJavaSerializableClass
import spark.rdd.{SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD}
import spark.util.MutablePair
class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
test("groupByKey without compression") {
@ -46,12 +49,12 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
val a = sc.parallelize(1 to 10, 2)
val b = a.map { x =>
(x, new ShuffleSuite.NonJavaSerializableClass(x * 2))
(x, new NonJavaSerializableClass(x * 2))
}
// If the Kryo serializer is not used correctly, the shuffle would fail because the
// default Java serializer cannot handle the non serializable class.
val c = new ShuffledRDD(b, new HashPartitioner(NUM_BLOCKS))
.setSerializer(classOf[spark.KryoSerializer].getName)
val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)](
b, new HashPartitioner(NUM_BLOCKS)).setSerializer(classOf[spark.KryoSerializer].getName)
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId
assert(c.count === 10)
@ -68,12 +71,12 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
sc = new SparkContext("local-cluster[2,1,512]", "test")
val a = sc.parallelize(1 to 10, 2)
val b = a.map { x =>
(x, new ShuffleSuite.NonJavaSerializableClass(x * 2))
(x, new NonJavaSerializableClass(x * 2))
}
// If the Kryo serializer is not used correctly, the shuffle would fail because the
// default Java serializer cannot handle the non serializable class.
val c = new ShuffledRDD(b, new HashPartitioner(3))
.setSerializer(classOf[spark.KryoSerializer].getName)
val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)](
b, new HashPartitioner(3)).setSerializer(classOf[spark.KryoSerializer].getName)
assert(c.count === 10)
}
@ -88,7 +91,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
// NOTE: The default Java serializer doesn't create zero-sized blocks.
// So, use Kryo
val c = new ShuffledRDD(b, new HashPartitioner(10))
val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10))
.setSerializer(classOf[spark.KryoSerializer].getName)
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId
@ -114,7 +117,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
val b = a.map(x => (x, x*2))
// NOTE: The default Java serializer should create zero-sized blocks
val c = new ShuffledRDD(b, new HashPartitioner(10))
val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10))
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId
assert(c.count === 4)
@ -128,6 +131,72 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
// We should have at most 4 non-zero sized partitions
assert(nonEmptyBlocks.size <= 4)
}
test("shuffle using mutable pairs") {
// Use a local cluster with 2 processes to make sure there are both local and remote blocks
sc = new SparkContext("local-cluster[2,1,512]", "test")
def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2)
val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1))
val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2)
val results = new ShuffledRDD[Int, Int, MutablePair[Int, Int]](pairs, new HashPartitioner(2))
.collect()
data.foreach { pair => results should contain (pair) }
}
test("sorting using mutable pairs") {
// This is not in SortingSuite because of the local cluster setup.
// Use a local cluster with 2 processes to make sure there are both local and remote blocks
sc = new SparkContext("local-cluster[2,1,512]", "test")
def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2)
val data = Array(p(1, 11), p(3, 33), p(100, 100), p(2, 22))
val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2)
val results = new OrderedRDDFunctions[Int, Int, MutablePair[Int, Int]](pairs)
.sortByKey().collect()
results(0) should be (p(1, 11))
results(1) should be (p(2, 22))
results(2) should be (p(3, 33))
results(3) should be (p(100, 100))
}
test("cogroup using mutable pairs") {
// Use a local cluster with 2 processes to make sure there are both local and remote blocks
sc = new SparkContext("local-cluster[2,1,512]", "test")
def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2)
val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1))
val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22"), p(3, "3"))
val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2)
val pairs2: RDD[MutablePair[Int, String]] = sc.parallelize(data2, 2)
val results = new CoGroupedRDD[Int](Seq(pairs1, pairs2), new HashPartitioner(2)).collectAsMap()
assert(results(1)(0).length === 3)
assert(results(1)(0).contains(1))
assert(results(1)(0).contains(2))
assert(results(1)(0).contains(3))
assert(results(1)(1).length === 2)
assert(results(1)(1).contains("11"))
assert(results(1)(1).contains("12"))
assert(results(2)(0).length === 1)
assert(results(2)(0).contains(1))
assert(results(2)(1).length === 1)
assert(results(2)(1).contains("22"))
assert(results(3)(0).length === 0)
assert(results(3)(1).contains("3"))
}
test("subtract mutable pairs") {
// Use a local cluster with 2 processes to make sure there are both local and remote blocks
sc = new SparkContext("local-cluster[2,1,512]", "test")
def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2)
val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1), p(3, 33))
val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22"))
val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2)
val pairs2: RDD[MutablePair[Int, String]] = sc.parallelize(data2, 2)
val results = new SubtractedRDD(pairs1, pairs2, new HashPartitioner(2)).collect()
results should have length (1)
// substracted rdd return results as Tuple2
results(0) should be ((3, 33))
}
}
object ShuffleSuite {

View file

@ -59,7 +59,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
override def stop() = {}
override def submitTasks(taskSet: TaskSet) = {
// normally done by TaskSetManager
taskSet.tasks.foreach(_.generation = mapOutputTracker.getGeneration)
taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch)
taskSets += taskSet
}
override def setListener(listener: TaskSchedulerListener) = {}
@ -299,10 +299,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
val reduceRdd = makeRdd(2, List(shuffleDep))
submit(reduceRdd, Array(0, 1))
// pretend we were told hostA went away
val oldGeneration = mapOutputTracker.getGeneration
val oldEpoch = mapOutputTracker.getEpoch
runEvent(ExecutorLost("exec-hostA"))
val newGeneration = mapOutputTracker.getGeneration
assert(newGeneration > oldGeneration)
val newEpoch = mapOutputTracker.getEpoch
assert(newEpoch > oldEpoch)
val noAccum = Map[Long, Any]()
val taskSet = taskSets(0)
// should be ignored for being too old
@ -311,8 +311,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null, null))
// should be ignored for being too old
runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null))
// should work because it's a new generation
taskSet.tasks(1).generation = newGeneration
// should work because it's a new epoch
taskSet.tasks(1).epoch = newEpoch
runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null, null))
assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) ===
Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA")))
@ -401,12 +401,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
assert(results === Map(0 -> 42))
}
/** Assert that the supplied TaskSet has exactly the given preferredLocations. Note, converts taskSet's locations to host only. */
private def assertLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) {
assert(locations.size === taskSet.tasks.size)
for ((expectLocs, taskLocs) <-
taskSet.tasks.map(_.preferredLocations).zip(locations)) {
assert(expectLocs.map(loc => spark.Utils.parseHostPort(loc)._1) === taskLocs)
/**
* Assert that the supplied TaskSet has exactly the given hosts as its preferred locations.
* Note that this checks only the host and not the executor ID.
*/
private def assertLocations(taskSet: TaskSet, hosts: Seq[Seq[String]]) {
assert(hosts.size === taskSet.tasks.size)
for ((taskLocs, expectedLocs) <- taskSet.tasks.map(_.preferredLocations).zip(hosts)) {
assert(taskLocs.map(_.host) === expectedLocs)
}
}

View file

@ -15,7 +15,7 @@
* limitations under the License.
*/
package spark.scheduler
package spark.scheduler.cluster
import org.scalatest.FunSuite
import org.scalatest.BeforeAndAfter
@ -27,7 +27,7 @@ import scala.collection.mutable.ArrayBuffer
import java.util.Properties
class DummyTaskSetManager(
class FakeTaskSetManager(
initPriority: Int,
initStageId: Int,
initNumTasks: Int,
@ -72,10 +72,16 @@ class DummyTaskSetManager(
override def executorLost(executorId: String, host: String): Unit = {
}
override def slaveOffer(execId: String, host: String, avaiableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = {
override def resourceOffer(
execId: String,
host: String,
availableCpus: Int,
maxLocality: TaskLocality.TaskLocality)
: Option[TaskDescription] =
{
if (tasksFinished + runningTasks < numTasks) {
increaseRunningTasks(1)
return Some(new TaskDescription(0, execId, "task 0:0", null))
return Some(new TaskDescription(0, execId, "task 0:0", 0, null))
}
return None
}
@ -98,17 +104,10 @@ class DummyTaskSetManager(
}
}
class DummyTask(stageId: Int) extends Task[Int](stageId)
{
def run(attemptId: Long): Int = {
return 0
}
}
class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging {
def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: ClusterScheduler, taskSet: TaskSet): DummyTaskSetManager = {
new DummyTaskSetManager(priority, stage, numTasks, cs , taskSet)
def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: ClusterScheduler, taskSet: TaskSet): FakeTaskSetManager = {
new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet)
}
def resourceOffer(rootPool: Pool): Int = {
@ -118,7 +117,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format(manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks))
}
for (taskSet <- taskSetQueue) {
taskSet.slaveOffer("execId_1", "hostname_1", 1) match {
taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match {
case Some(task) =>
return taskSet.stageId
case None => {}
@ -135,7 +134,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
sc = new SparkContext("local", "ClusterSchedulerSuite")
val clusterScheduler = new ClusterScheduler(sc)
var tasks = ArrayBuffer[Task[_]]()
val task = new DummyTask(0)
val task = new FakeTask(0)
tasks += task
val taskSet = new TaskSet(tasks.toArray,0,0,0,null)
@ -162,7 +161,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
sc = new SparkContext("local", "ClusterSchedulerSuite")
val clusterScheduler = new ClusterScheduler(sc)
var tasks = ArrayBuffer[Task[_]]()
val task = new DummyTask(0)
val task = new FakeTask(0)
tasks += task
val taskSet = new TaskSet(tasks.toArray,0,0,0,null)
@ -219,7 +218,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
sc = new SparkContext("local", "ClusterSchedulerSuite")
val clusterScheduler = new ClusterScheduler(sc)
var tasks = ArrayBuffer[Task[_]]()
val task = new DummyTask(0)
val task = new FakeTask(0)
tasks += task
val taskSet = new TaskSet(tasks.toArray,0,0,0,null)

View file

@ -0,0 +1,273 @@
/*
* 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 spark.scheduler.cluster
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable
import org.scalatest.FunSuite
import spark._
import spark.scheduler._
import spark.executor.TaskMetrics
import java.nio.ByteBuffer
import spark.util.FakeClock
/**
* A mock ClusterScheduler implementation that just remembers information about tasks started and
* feedback received from the TaskSetManagers. Note that it's important to initialize this with
* a list of "live" executors and their hostnames for isExecutorAlive and hasExecutorsAliveOnHost
* to work, and these are required for locality in ClusterTaskSetManager.
*/
class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */)
extends ClusterScheduler(sc)
{
val startedTasks = new ArrayBuffer[Long]
val endedTasks = new mutable.HashMap[Long, TaskEndReason]
val finishedManagers = new ArrayBuffer[TaskSetManager]
val executors = new mutable.HashMap[String, String] ++ liveExecutors
listener = new TaskSchedulerListener {
def taskStarted(task: Task[_], taskInfo: TaskInfo) {
startedTasks += taskInfo.index
}
def taskEnded(
task: Task[_],
reason: TaskEndReason,
result: Any,
accumUpdates: mutable.Map[Long, Any],
taskInfo: TaskInfo,
taskMetrics: TaskMetrics)
{
endedTasks(taskInfo.index) = reason
}
def executorGained(execId: String, host: String) {}
def executorLost(execId: String) {}
def taskSetFailed(taskSet: TaskSet, reason: String) {}
}
def removeExecutor(execId: String): Unit = executors -= execId
override def taskSetFinished(manager: TaskSetManager): Unit = finishedManagers += manager
override def isExecutorAlive(execId: String): Boolean = executors.contains(execId)
override def hasExecutorsAliveOnHost(host: String): Boolean = executors.values.exists(_ == host)
}
class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL}
val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong
test("TaskSet with no preferences") {
sc = new SparkContext("local", "test")
val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
val taskSet = createTaskSet(1)
val manager = new ClusterTaskSetManager(sched, taskSet)
// Offer a host with no CPUs
assert(manager.resourceOffer("exec1", "host1", 0, ANY) === None)
// Offer a host with process-local as the constraint; this should work because the TaskSet
// above won't have any locality preferences
val taskOption = manager.resourceOffer("exec1", "host1", 2, TaskLocality.PROCESS_LOCAL)
assert(taskOption.isDefined)
val task = taskOption.get
assert(task.executorId === "exec1")
assert(sched.startedTasks.contains(0))
// Re-offer the host -- now we should get no more tasks
assert(manager.resourceOffer("exec1", "host1", 2, PROCESS_LOCAL) === None)
// Tell it the task has finished
manager.statusUpdate(0, TaskState.FINISHED, createTaskResult(0))
assert(sched.endedTasks(0) === Success)
assert(sched.finishedManagers.contains(manager))
}
test("multiple offers with no preferences") {
sc = new SparkContext("local", "test")
val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
val taskSet = createTaskSet(3)
val manager = new ClusterTaskSetManager(sched, taskSet)
// First three offers should all find tasks
for (i <- 0 until 3) {
val taskOption = manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL)
assert(taskOption.isDefined)
val task = taskOption.get
assert(task.executorId === "exec1")
}
assert(sched.startedTasks.toSet === Set(0, 1, 2))
// Re-offer the host -- now we should get no more tasks
assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
// Finish the first two tasks
manager.statusUpdate(0, TaskState.FINISHED, createTaskResult(0))
manager.statusUpdate(1, TaskState.FINISHED, createTaskResult(1))
assert(sched.endedTasks(0) === Success)
assert(sched.endedTasks(1) === Success)
assert(!sched.finishedManagers.contains(manager))
// Finish the last task
manager.statusUpdate(2, TaskState.FINISHED, createTaskResult(2))
assert(sched.endedTasks(2) === Success)
assert(sched.finishedManagers.contains(manager))
}
test("basic delay scheduling") {
sc = new SparkContext("local", "test")
val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2"))
val taskSet = createTaskSet(4,
Seq(TaskLocation("host1", "exec1")),
Seq(TaskLocation("host2", "exec2")),
Seq(TaskLocation("host1"), TaskLocation("host2", "exec2")),
Seq() // Last task has no locality prefs
)
val clock = new FakeClock
val manager = new ClusterTaskSetManager(sched, taskSet, clock)
// First offer host1, exec1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
// Offer host1, exec1 again: the last task, which has no prefs, should be chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 3)
// Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen
assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
clock.advance(LOCALITY_WAIT)
// Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen
assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
// Offer host1, exec1 again, at NODE_LOCAL level: we should choose task 2
assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL).get.index == 2)
// Offer host1, exec1 again, at NODE_LOCAL level: nothing should get chosen
assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL) === None)
// Offer host1, exec1 again, at ANY level: nothing should get chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
clock.advance(LOCALITY_WAIT)
// Offer host1, exec1 again, at ANY level: task 1 should get chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1)
// Offer host1, exec1 again, at ANY level: nothing should be chosen as we've launched all tasks
assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
}
test("delay scheduling with fallback") {
sc = new SparkContext("local", "test")
val sched = new FakeClusterScheduler(sc,
("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3"))
val taskSet = createTaskSet(5,
Seq(TaskLocation("host1")),
Seq(TaskLocation("host2")),
Seq(TaskLocation("host2")),
Seq(TaskLocation("host3")),
Seq(TaskLocation("host2"))
)
val clock = new FakeClock
val manager = new ClusterTaskSetManager(sched, taskSet, clock)
// First offer host1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
// Offer host1 again: nothing should get chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
clock.advance(LOCALITY_WAIT)
// Offer host1 again: second task (on host2) should get chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1)
// Offer host1 again: third task (on host2) should get chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2)
// Offer host2: fifth task (also on host2) should get chosen
assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 4)
// Now that we've launched a local task, we should no longer launch the task for host3
assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None)
clock.advance(LOCALITY_WAIT)
// After another delay, we can go ahead and launch that task non-locally
assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 3)
}
test("delay scheduling with failed hosts") {
sc = new SparkContext("local", "test")
val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2"))
val taskSet = createTaskSet(3,
Seq(TaskLocation("host1")),
Seq(TaskLocation("host2")),
Seq(TaskLocation("host3"))
)
val clock = new FakeClock
val manager = new ClusterTaskSetManager(sched, taskSet, clock)
// First offer host1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
// Offer host1 again: third task should be chosen immediately because host3 is not up
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2)
// After this, nothing should get chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
// Now mark host2 as dead
sched.removeExecutor("exec2")
manager.executorLost("exec2", "host2")
// Task 1 should immediately be launched on host1 because its original host is gone
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1)
// Now that all tasks have launched, nothing new should be launched anywhere else
assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None)
}
/**
* Utility method to create a TaskSet, potentially setting a particular sequence of preferred
* locations for each task (given as varargs) if this sequence is not empty.
*/
def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = {
if (prefLocs.size != 0 && prefLocs.size != numTasks) {
throw new IllegalArgumentException("Wrong number of task locations")
}
val tasks = Array.tabulate[Task[_]](numTasks) { i =>
new FakeTask(i, if (prefLocs.size != 0) prefLocs(i) else Nil)
}
new TaskSet(tasks, 0, 0, 0, null)
}
def createTaskResult(id: Int): ByteBuffer = {
ByteBuffer.wrap(Utils.serialize(new TaskResult[Int](id, mutable.Map.empty, new TaskMetrics)))
}
}

View file

@ -15,13 +15,12 @@
* limitations under the License.
*/
package org.apache.hadoop.mapred
package spark.scheduler.cluster
trait HadoopMapRedUtil {
def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
import spark.scheduler.{TaskLocation, Task}
def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId) {
override def run(attemptId: Long): Int = 0
def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier,
jobId, isMap, taskId, attemptId)
override def preferredLocations: Seq[TaskLocation] = prefLocs
}

View file

@ -15,7 +15,7 @@
* limitations under the License.
*/
package spark.scheduler
package spark.scheduler.local
import org.scalatest.FunSuite
import org.scalatest.BeforeAndAfter

View file

@ -24,14 +24,15 @@ import org.eclipse.jetty.server.Server
class UISuite extends FunSuite {
test("jetty port increases under contention") {
val startPort = 33333
val startPort = 3030
val server = new Server(startPort)
server.start()
val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq())
val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("localhost", startPort, Seq())
assert(boundPort1 === startPort + 1)
assert(boundPort2 === startPort + 2)
// Allow some wiggle room in case ports on the machine are under contention
assert(boundPort1 > startPort && boundPort1 < startPort + 10)
assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10)
}
test("jetty binds to port 0 correctly") {

View file

@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package spark.util
class FakeClock extends Clock {
private var time = 0L
def advance(millis: Long): Unit = time += millis
def getTime(): Long = time
}

View file

@ -18,7 +18,7 @@
require 'fileutils'
include FileUtils
if ENV['SKIP_API'] != '1'
if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1')
# Build Scaladoc for Java/Scala
projects = ["core", "examples", "repl", "bagel", "streaming", "mllib"]

View file

@ -8,22 +8,26 @@ title: Building Spark with Maven
Building Spark using Maven Requires Maven 3 (the build process is tested with Maven 3.0.4) and Java 1.6 or newer.
Building with Maven requires that a Hadoop profile be specified explicitly at the command line, there is no default. There are two profiles to choose from, one for building for Hadoop 1 or Hadoop 2.
## Specifying the Hadoop version ##
for Hadoop 1 (using 0.20.205.0) use:
To enable support for HDFS and other Hadoop-supported storage systems, specify the exact Hadoop version by setting the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default.
$ mvn -Phadoop1 clean install
For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use:
# Apache Hadoop 1.2.1
$ mvn -Dhadoop.version=1.2.1 clean install
for Hadoop 2 (using 2.0.0-mr1-cdh4.1.1) use:
# Cloudera CDH 4.2.0 with MapReduce v1
$ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 clean install
$ mvn -Phadoop2 clean install
For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, enable the "hadoop2-yarn" profile:
It uses the scala-maven-plugin which supports incremental and continuous compilation. E.g.
# Apache Hadoop 2.0.5-alpha
$ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha clean install
$ mvn -Phadoop2 scala:cc
# Cloudera CDH 4.2.0 with MapReduce v2
$ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 clean install
…should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively.
## Spark Tests in Maven ##
@ -31,11 +35,11 @@ Tests are run by default via the scalatest-maven-plugin. With this you can do th
Skip test execution (but not compilation):
$ mvn -DskipTests -Phadoop2 clean install
$ mvn -Dhadoop.version=... -DskipTests clean install
To run a specific test suite:
$ mvn -Phadoop2 -Dsuites=spark.repl.ReplSuite test
$ mvn -Dhadoop.version=... -Dsuites=spark.repl.ReplSuite test
## Setting up JVM Memory Usage Via Maven ##
@ -53,6 +57,15 @@ To fix these, you can do the following:
export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=128M"
## Continuous Compilation ##
We use the scala-maven-plugin which supports incremental and continuous compilation. E.g.
$ mvn scala:cc
…should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively.
## Using With IntelliJ IDEA ##
This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the pom.xml file in the project root folder, you only need to activate either the hadoop1 or hadoop2 profile in the "Maven Properties" popout. We have not tried Eclipse/Scala IDE with this.
@ -61,6 +74,6 @@ This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the
It includes support for building a Debian package containing a 'fat-jar' which includes the repl, the examples and bagel. This can be created by specifying the deb profile:
$ mvn -Phadoop2,deb clean install
$ mvn -Pdeb clean install
The debian package can then be found under repl/target. We added the short commit hash to the file name so that we can distinguish individual packages build for SNAPSHOT versions.

View file

@ -146,7 +146,7 @@ Apart from these, the following properties are also available, and may be useful
</tr>
<tr>
<td>spark.ui.port</td>
<td>33000</td>
<td>3030</td>
<td>
Port for your application's dashboard, which shows memory and workload data
</td>
@ -243,8 +243,34 @@ Apart from these, the following properties are also available, and may be useful
<td>3000</td>
<td>
Number of milliseconds to wait to launch a data-local task before giving up and launching it
in a non-data-local location. You should increase this if your tasks are long and you are seeing
poor data locality, but the default generally works well.
on a less-local node. The same wait will be used to step through multiple locality levels
(process-local, node-local, rack-local and then any). It is also possible to customize the
waiting time for each level by setting <code>spark.locality.wait.node</code>, etc.
You should increase this setting if your tasks are long and see poor locality, but the
default usually works well.
</td>
</tr>
<tr>
<td>spark.locality.wait.process</td>
<td>spark.locality.wait</td>
<td>
Customize the locality wait for process locality. This affects tasks that attempt to access
cached data in a particular executor process.
</td>
</tr>
<tr>
<td>spark.locality.wait.node</td>
<td>spark.locality.wait</td>
<td>
Customize the locality wait for node locality. For example, you can set this to 0 to skip
node locality and search immediately for rack locality (if your cluster has rack information).
</td>
</tr>
<tr>
<td>spark.locality.wait.rack</td>
<td>spark.locality.wait</td>
<td>
Customize the locality wait for rack locality.
</td>
</tr>
<tr>

View file

@ -4,10 +4,11 @@ title: Running Spark on EC2
---
The `spark-ec2` script, located in Spark's `ec2` directory, allows you
to launch, manage and shut down Spark clusters on Amazon EC2. It automatically sets up Mesos, Spark and HDFS
on the cluster for you.
This guide describes how to use `spark-ec2` to launch clusters, how to run jobs on them, and how to shut them down.
It assumes you've already signed up for an EC2 account on the [Amazon Web Services site](http://aws.amazon.com/).
to launch, manage and shut down Spark clusters on Amazon EC2. It automatically
sets up Spark, Shark and HDFS on the cluster for you. This guide describes
how to use `spark-ec2` to launch clusters, how to run jobs on them, and how
to shut them down. It assumes you've already signed up for an EC2 account
on the [Amazon Web Services site](http://aws.amazon.com/).
`spark-ec2` is designed to manage multiple named clusters. You can
launch a new cluster (telling the script its size and giving it a name),
@ -59,18 +60,22 @@ RAM). Refer to the Amazon pages about [EC2 instance
types](http://aws.amazon.com/ec2/instance-types) and [EC2
pricing](http://aws.amazon.com/ec2/#pricing) for information about other
instance types.
- `--region=<EC2_REGION>` specifies an EC2 region in which to launch
instances. The default region is `us-east-1`.
- `--zone=<EC2_ZONE>` can be used to specify an EC2 availability zone
to launch instances in. Sometimes, you will get an error because there
is not enough capacity in one zone, and you should try to launch in
another. This happens mostly with the `m1.large` instance types;
extra-large (both `m1.xlarge` and `c1.xlarge`) instances tend to be more
available.
another.
- `--ebs-vol-size=GB` will attach an EBS volume with a given amount
of space to each node so that you can have a persistent HDFS cluster
on your nodes across cluster restarts (see below).
- `--spot-price=PRICE` will launch the worker nodes as
[Spot Instances](http://aws.amazon.com/ec2/spot-instances/),
bidding for the given maximum price (in dollars).
- `--spark-version=VERSION` will pre-load the cluster with the
specified version of Spark. VERSION can be a version number
(e.g. "0.7.3") or a specific git hash. By default, a recent
version will be used.
- If one of your launches fails due to e.g. not having the right
permissions on your private key file, you can run `launch` with the
`--resume` option to restart the setup process on an existing cluster.
@ -99,9 +104,8 @@ permissions on your private key file, you can run `launch` with the
`spark-ec2` to attach a persistent EBS volume to each node for
storing the persistent HDFS.
- Finally, if you get errors while running your jobs, look at the slave's logs
for that job inside of the Mesos work directory (/mnt/mesos-work). You can
also view the status of the cluster using the Mesos web UI
(`http://<master-hostname>:8080`).
for that job inside of the scheduler work directory (/root/spark/work). You can
also view the status of the cluster using the web UI: `http://<master-hostname>:8080`.
# Configuration
@ -140,22 +144,14 @@ section.
# Limitations
- `spark-ec2` currently only launches machines in the US-East region of EC2.
It should not be hard to make it launch VMs in other zones, but you will need
to create your own AMIs in them.
- Support for "cluster compute" nodes is limited -- there's no way to specify a
locality group. However, you can launch slave nodes in your
`<clusterName>-slaves` group manually and then use `spark-ec2 launch
--resume` to start a cluster with them.
- Support for spot instances is limited.
If you have a patch or suggestion for one of these limitations, feel free to
[contribute](contributing-to-spark.html) it!
# Using a Newer Spark Version
The Spark EC2 machine images may not come with the latest version of Spark. To use a newer version, you can run `git pull` to pull in `/root/spark` to pull in the latest version of Spark from `git`, and build it using `sbt/sbt compile`. You will also need to copy it to all the other nodes in the cluster using `~/spark-ec2/copy-dir /root/spark`.
# Accessing Data in S3
Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n://<bucket>/path`. You will also need to set your Amazon security credentials, either by setting the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` before your program or through `SparkContext.hadoopConfiguration`. Full instructions on S3 access using the Hadoop input libraries can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3).

View file

@ -6,7 +6,7 @@ title: Launching Spark on YARN
Experimental support for running over a [YARN (Hadoop
NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html)
cluster was added to Spark in version 0.6.0. This was merged into master as part of 0.7 effort.
To build spark core with YARN support, please use the hadoop2-yarn profile.
To build spark with YARN support, please use the hadoop2-yarn profile.
Ex: mvn -Phadoop2-yarn clean install
# Building spark core consolidated jar.
@ -15,18 +15,12 @@ We need a consolidated spark core jar (which bundles all the required dependenci
This can be built either through sbt or via maven.
- Building spark assembled jar via sbt.
It is a manual process of enabling it in project/SparkBuild.scala.
Please comment out the
HADOOP_VERSION, HADOOP_MAJOR_VERSION and HADOOP_YARN
variables before the line 'For Hadoop 2 YARN support'
Next, uncomment the subsequent 3 variable declaration lines (for these three variables) which enable hadoop yarn support.
Enable YARN support by setting `SPARK_WITH_YARN=true` when invoking sbt:
Assembly of the jar Ex:
./sbt/sbt clean assembly
SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true ./sbt/sbt clean assembly
The assembled jar would typically be something like :
`./core/target/spark-core-assembly-0.8.0-SNAPSHOT.jar`
`./yarn/target/spark-yarn-assembly-0.8.0-SNAPSHOT.jar`
- Building spark assembled jar via Maven.
@ -34,16 +28,16 @@ The assembled jar would typically be something like :
Something like this. Ex:
mvn -Phadoop2-yarn clean package -DskipTests=true
mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha clean package -DskipTests=true
This will build the shaded (consolidated) jar. Typically something like :
`./repl-bin/target/spark-repl-bin-<VERSION>-shaded-hadoop2-yarn.jar`
`./yarn/target/spark-yarn-bin-<VERSION>-shaded.jar`
# Preparations
- Building spark core assembled jar (see above).
- Building spark-yarn assembly (see above).
- Your application code must be packaged into a separate JAR file.
If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt package`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different.
@ -55,7 +49,7 @@ This would be used to connect to the cluster, write to the dfs and submit jobs t
The command to launch the YARN Client is as follows:
SPARK_JAR=<SPARK_YAR_FILE> ./run spark.deploy.yarn.Client \
SPARK_JAR=<SPARK_YARN_JAR_FILE> ./run spark.deploy.yarn.Client \
--jar <YOUR_APP_JAR_FILE> \
--class <APP_MAIN_CLASS> \
--args <APP_MAIN_ARGUMENTS> \
@ -68,7 +62,7 @@ The command to launch the YARN Client is as follows:
For example:
SPARK_JAR=./core/target/spark-core-assembly-{{site.SPARK_VERSION}}.jar ./run spark.deploy.yarn.Client \
SPARK_JAR=./yarn/target/spark-yarn-assembly-{{site.SPARK_VERSION}}.jar ./run spark.deploy.yarn.Client \
--jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}.jar \
--class spark.examples.SparkPi \
--args yarn-standalone \

View file

@ -7,11 +7,46 @@ A "Spark Streaming" receiver can be a simple network stream, streams of messages
This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application.
## A quick and naive walk-through
### Write a simple receiver
This starts with implementing [NetworkReceiver](#References)
Following is a simple socket text-stream receiver.
{% highlight scala %}
class SocketTextStreamReceiver(host: String,
port: Int
) extends NetworkReceiver[String] {
protected lazy val blocksGenerator: BlockGenerator =
new BlockGenerator(StorageLevel.MEMORY_ONLY_SER_2)
protected def onStart() = {
blocksGenerator.start()
val socket = new Socket(host, port)
val dataInputStream = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8"))
var data: String = dataInputStream.readLine()
while (data != null) {
blocksGenerator += data
data = dataInputStream.readLine()
}
}
protected def onStop() {
blocksGenerator.stop()
}
}
{% endhighlight %}
All we did here is extended NetworkReceiver and called blockGenerator's API method (i.e. +=) to push our blocks of data. Please refer to scala-docs of NetworkReceiver for more details.
### An Actor as Receiver.
This starts with implementing [Actor](#References)
Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api.
@ -46,7 +81,16 @@ All we did here is mixed in trait Receiver and called pushBlock api method to pu
{% endhighlight %}
* Plug-in the actor configuration into the spark streaming context and create a DStream.
* Plug-in the custom receiver into the spark streaming context and create a DStream.
{% highlight scala %}
val lines = ssc.networkStream[String](new SocketTextStreamReceiver(
"localhost", 8445))
{% endhighlight %}
* OR Plug-in the actor as receiver into the spark streaming context and create a DStream.
{% highlight scala %}
@ -99,3 +143,4 @@ _A more comprehensive example is provided in the spark streaming examples_
## References
1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html)
2.[NetworkReceiver](http://spark-project.org/docs/latest/api/streaming/index.html#spark.streaming.dstream.NetworkReceiver)

View file

@ -301,6 +301,9 @@ dstream.checkpoint(checkpointInterval) // checkpointInterval must be a multiple
For DStreams that must be checkpointed (that is, DStreams created by `updateStateByKey` and `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds.
## Customizing Receiver
Spark comes with a built in support for most common usage scenarios where input stream source can be either a network socket stream to support for a few message queues. Apart from that it is also possible to supply your own custom receiver via a convenient API. Find more details at [Custom Receiver Guide](streaming-custom-receivers.html)
# Performance Tuning
Getting the best performance of a Spark Streaming application on a cluster requires a bit of tuning. This section explains a number of the parameters and configurations that can tuned to improve the performance of you application. At a high level, you need to consider two things:
<ol>

View file

@ -1,9 +0,0 @@
#!/usr/bin/env bash
# These variables are automatically filled in by the mesos-ec2 script.
export MESOS_MASTERS="{{master_list}}"
export MESOS_SLAVES="{{slave_list}}"
export MESOS_ZOO_LIST="{{zoo_list}}"
export MESOS_HDFS_DATA_DIRS="{{hdfs_data_dirs}}"
export MESOS_MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}"
export MESOS_SPARK_LOCAL_DIRS="{{spark_local_dirs}}"

View file

@ -1,11 +1,13 @@
#!/usr/bin/env bash
# These variables are automatically filled in by the mesos-ec2 script.
export MESOS_MASTERS="{{master_list}}"
export MESOS_SLAVES="{{slave_list}}"
export MESOS_ZOO_LIST="{{zoo_list}}"
export MESOS_HDFS_DATA_DIRS="{{hdfs_data_dirs}}"
export MESOS_MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}"
export MESOS_SPARK_LOCAL_DIRS="{{spark_local_dirs}}"
# These variables are automatically filled in by the spark-ec2 script.
export MASTERS="{{master_list}}"
export SLAVES="{{slave_list}}"
export HDFS_DATA_DIRS="{{hdfs_data_dirs}}"
export MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}"
export SPARK_LOCAL_DIRS="{{spark_local_dirs}}"
export MODULES="{{modules}}"
export SPARK_VERSION="{{spark_version}}"
export SHARK_VERSION="{{shark_version}}"
export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}"
export SWAP_MB="{{swap}}"

View file

@ -36,9 +36,8 @@ import boto
from boto.ec2.blockdevicemapping import BlockDeviceMapping, EBSBlockDeviceType
from boto import ec2
# A static URL from which to figure out the latest Mesos EC2 AMI
LATEST_AMI_URL = "https://s3.amazonaws.com/mesos-images/ids/latest-spark-0.7"
# A URL prefix from which to fetch AMI information
AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list"
# Configure and parse our command-line arguments
def parse_args():
@ -66,9 +65,14 @@ def parse_args():
help="Availability zone to launch instances in, or 'all' to spread " +
"slaves across multiple (an additional $0.01/Gb for bandwidth" +
"between zones applies)")
parser.add_option("-a", "--ami", default="latest",
help="Amazon Machine Image ID to use, or 'latest' to use latest " +
"available AMI (default: latest)")
parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use")
parser.add_option("-v", "--spark-version", default="0.7.3",
help="Version of Spark to use: 'X.Y.Z' or a specific git hash")
parser.add_option("--spark-git-repo",
default="https://github.com/mesos/spark",
help="Github repo from which to checkout supplied commit hash")
parser.add_option("--hadoop-major-version", default="1",
help="Major version of Hadoop (default: 1)")
parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port",
help="Use SSH dynamic port forwarding to create a SOCKS proxy at " +
"the given local address (for use with login)")
@ -84,17 +88,11 @@ def parse_args():
parser.add_option("--spot-price", metavar="PRICE", type="float",
help="If specified, launch slaves as spot instances with the given " +
"maximum price (in dollars)")
parser.add_option("--cluster-type", type="choice", metavar="TYPE",
choices=["mesos", "standalone"], default="standalone",
help="'mesos' for a Mesos cluster, 'standalone' for a standalone " +
"Spark cluster (default: standalone)")
parser.add_option("--ganglia", action="store_true", default=True,
help="Setup Ganglia monitoring on cluster (default: on). NOTE: " +
"the Ganglia page will be publicly accessible")
parser.add_option("--no-ganglia", action="store_false", dest="ganglia",
help="Disable Ganglia monitoring for the cluster")
parser.add_option("--old-scripts", action="store_true", default=False,
help="Use old mesos-ec2 scripts, for Spark <= 0.6 AMIs")
parser.add_option("-u", "--user", default="root",
help="The SSH user you want to connect as (default: root)")
parser.add_option("--delete-groups", action="store_true", default=False,
@ -109,9 +107,6 @@ def parse_args():
print >> stderr, ("ERROR: The -i or --identity-file argument is " +
"required for " + action)
sys.exit(1)
if opts.cluster_type not in ["mesos", "standalone"] and action == "launch":
print >> stderr, ("ERROR: Invalid cluster type: " + opts.cluster_type)
sys.exit(1)
# Boto config check
# http://boto.cloudhackers.com/en/latest/boto_config_tut.html
@ -158,67 +153,96 @@ def wait_for_instances(conn, instances):
def is_active(instance):
return (instance.state in ['pending', 'running', 'stopping', 'stopped'])
# Return correct versions of Spark and Shark, given the supplied Spark version
def get_spark_shark_version(opts):
spark_shark_map = {"0.7.3": "0.7.0"}
version = opts.spark_version.replace("v", "")
if version not in spark_shark_map:
print >> stderr, "Don't know about Spark version: %s" % version
sys.exit(1)
return (version, spark_shark_map[version])
# Attempt to resolve an appropriate AMI given the architecture and
# region of the request.
def get_spark_ami(opts):
instance_types = {
"m1.small": "pvm",
"m1.medium": "pvm",
"m1.large": "pvm",
"m1.xlarge": "pvm",
"t1.micro": "pvm",
"c1.medium": "pvm",
"c1.xlarge": "pvm",
"m2.xlarge": "pvm",
"m2.2xlarge": "pvm",
"m2.4xlarge": "pvm",
"cc1.4xlarge": "hvm",
"cc2.8xlarge": "hvm",
"cg1.4xlarge": "hvm",
"hs1.8xlarge": "hvm",
"hi1.4xlarge": "hvm",
"m3.xlarge": "hvm",
"m3.2xlarge": "hvm",
"cr1.8xlarge": "hvm"
}
if opts.instance_type in instance_types:
instance_type = instance_types[opts.instance_type]
else:
instance_type = "pvm"
print >> stderr,\
"Don't recognize %s, assuming type is pvm" % opts.instance_type
ami_path = "%s/%s/%s" % (AMI_PREFIX, opts.region, instance_type)
try:
ami = urllib2.urlopen(ami_path).read().strip()
print "Spark AMI: " + ami
except:
print >> stderr, "Could not resolve AMI at: " + ami_path
sys.exit(1)
return ami
# Launch a cluster of the given name, by setting up its security groups,
# and then starting new instances in them.
# Returns a tuple of EC2 reservation objects for the master, slave
# and zookeeper instances (in that order).
# Returns a tuple of EC2 reservation objects for the master and slaves
# Fails if there already instances running in the cluster's groups.
def launch_cluster(conn, opts, cluster_name):
print "Setting up security groups..."
master_group = get_or_make_group(conn, cluster_name + "-master")
slave_group = get_or_make_group(conn, cluster_name + "-slaves")
zoo_group = get_or_make_group(conn, cluster_name + "-zoo")
if master_group.rules == []: # Group was just now created
master_group.authorize(src_group=master_group)
master_group.authorize(src_group=slave_group)
master_group.authorize(src_group=zoo_group)
master_group.authorize('tcp', 22, 22, '0.0.0.0/0')
master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0')
master_group.authorize('tcp', 33000, 33000, '0.0.0.0/0')
master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0')
master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0')
master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0')
master_group.authorize('tcp', 33000, 33010, '0.0.0.0/0')
if opts.cluster_type == "mesos":
master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0')
master_group.authorize('tcp', 3030, 3035, '0.0.0.0/0')
if opts.ganglia:
master_group.authorize('tcp', 5080, 5080, '0.0.0.0/0')
if slave_group.rules == []: # Group was just now created
slave_group.authorize(src_group=master_group)
slave_group.authorize(src_group=slave_group)
slave_group.authorize(src_group=zoo_group)
slave_group.authorize('tcp', 22, 22, '0.0.0.0/0')
slave_group.authorize('tcp', 8080, 8081, '0.0.0.0/0')
slave_group.authorize('tcp', 50060, 50060, '0.0.0.0/0')
slave_group.authorize('tcp', 50075, 50075, '0.0.0.0/0')
slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0')
slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0')
if zoo_group.rules == []: # Group was just now created
zoo_group.authorize(src_group=master_group)
zoo_group.authorize(src_group=slave_group)
zoo_group.authorize(src_group=zoo_group)
zoo_group.authorize('tcp', 22, 22, '0.0.0.0/0')
zoo_group.authorize('tcp', 2181, 2181, '0.0.0.0/0')
zoo_group.authorize('tcp', 2888, 2888, '0.0.0.0/0')
zoo_group.authorize('tcp', 3888, 3888, '0.0.0.0/0')
# Check if instances are already running in our groups
active_nodes = get_existing_cluster(conn, opts, cluster_name,
die_on_error=False)
if any(active_nodes):
print >> stderr, ("ERROR: There are already instances running in " +
"group %s, %s or %s" % (master_group.name, slave_group.name, zoo_group.name))
sys.exit(1)
# Figure out the latest AMI from our static URL
if opts.ami == "latest":
try:
opts.ami = urllib2.urlopen(LATEST_AMI_URL).read().strip()
print "Latest Spark AMI: " + opts.ami
except:
print >> stderr, "Could not read " + LATEST_AMI_URL
"group %s or %s" % (master_group.name, slave_group.name))
sys.exit(1)
# Figure out Spark AMI
if opts.ami is None:
opts.ami = get_spark_ami(opts)
print "Launching instances..."
try:
@ -285,9 +309,9 @@ def launch_cluster(conn, opts, cluster_name):
print "Canceling spot instance requests"
conn.cancel_spot_instance_requests(my_req_ids)
# Log a warning if any of these requests actually launched instances:
(master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(
(master_nodes, slave_nodes) = get_existing_cluster(
conn, opts, cluster_name, die_on_error=False)
running = len(master_nodes) + len(slave_nodes) + len(zoo_nodes)
running = len(master_nodes) + len(slave_nodes)
if running:
print >> stderr, ("WARNING: %d instances are still running" % running)
sys.exit(0)
@ -328,21 +352,17 @@ def launch_cluster(conn, opts, cluster_name):
master_nodes = master_res.instances
print "Launched master in %s, regid = %s" % (zone, master_res.id)
zoo_nodes = []
# Return all the instances
return (master_nodes, slave_nodes, zoo_nodes)
return (master_nodes, slave_nodes)
# Get the EC2 instances in an existing cluster if available.
# Returns a tuple of lists of EC2 instance objects for the masters,
# slaves and zookeeper nodes (in that order).
# Returns a tuple of lists of EC2 instance objects for the masters and slaves
def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
print "Searching for existing cluster " + cluster_name + "..."
reservations = conn.get_all_instances()
master_nodes = []
slave_nodes = []
zoo_nodes = []
for res in reservations:
active = [i for i in res.instances if is_active(i)]
if len(active) > 0:
@ -351,13 +371,11 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
master_nodes += res.instances
elif group_names == [cluster_name + "-slaves"]:
slave_nodes += res.instances
elif group_names == [cluster_name + "-zoo"]:
zoo_nodes += res.instances
if any((master_nodes, slave_nodes, zoo_nodes)):
print ("Found %d master(s), %d slaves, %d ZooKeeper nodes" %
(len(master_nodes), len(slave_nodes), len(zoo_nodes)))
if any((master_nodes, slave_nodes)):
print ("Found %d master(s), %d slaves" %
(len(master_nodes), len(slave_nodes)))
if (master_nodes != [] and slave_nodes != []) or not die_on_error:
return (master_nodes, slave_nodes, zoo_nodes)
return (master_nodes, slave_nodes)
else:
if master_nodes == [] and slave_nodes != []:
print "ERROR: Could not find master in group " + cluster_name + "-master"
@ -370,7 +388,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
# Deploy configuration files and run setup scripts on a newly launched
# or started EC2 cluster.
def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_key):
def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
master = master_nodes[0].public_dns_name
if deploy_ssh_key:
print "Copying SSH key %s to master..." % opts.identity_file
@ -378,38 +396,26 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k
scp(master, opts, opts.identity_file, '~/.ssh/id_rsa')
ssh(master, opts, 'chmod 600 ~/.ssh/id_rsa')
if opts.cluster_type == "mesos":
modules = ['ephemeral-hdfs', 'persistent-hdfs', 'mesos']
elif opts.cluster_type == "standalone":
modules = ['ephemeral-hdfs', 'persistent-hdfs', 'spark-standalone']
modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs',
'mapreduce', 'spark-standalone']
if opts.hadoop_major_version == "1":
modules = filter(lambda x: x != "mapreduce", modules)
if opts.ganglia:
modules.append('ganglia')
if not opts.old_scripts:
# NOTE: We should clone the repository before running deploy_files to
# prevent ec2-variables.sh from being overwritten
ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git")
ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git -b v2")
print "Deploying files to master..."
deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes,
zoo_nodes, modules)
deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, modules)
print "Running setup on master..."
if opts.old_scripts:
if opts.cluster_type == "mesos":
setup_mesos_cluster(master, opts)
elif opts.cluster_type == "standalone":
setup_standalone_cluster(master, slave_nodes, opts)
else:
setup_spark_cluster(master, opts)
print "Done!"
def setup_mesos_cluster(master, opts):
ssh(master, opts, "chmod u+x mesos-ec2/setup")
ssh(master, opts, "mesos-ec2/setup %s %s %s %s" %
("generic", "none", "master", opts.swap))
def setup_standalone_cluster(master, slave_nodes, opts):
slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes])
ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips))
@ -418,9 +424,6 @@ def setup_standalone_cluster(master, slave_nodes, opts):
def setup_spark_cluster(master, opts):
ssh(master, opts, "chmod u+x spark-ec2/setup.sh")
ssh(master, opts, "spark-ec2/setup.sh")
if opts.cluster_type == "mesos":
print "Mesos cluster started at http://%s:8080" % master
elif opts.cluster_type == "standalone":
print "Spark standalone cluster started at http://%s:8080" % master
if opts.ganglia:
@ -428,13 +431,11 @@ def setup_spark_cluster(master, opts):
# Wait for a whole cluster (masters, slaves and ZooKeeper) to start up
def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes, zoo_nodes):
def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes):
print "Waiting for instances to start up..."
time.sleep(5)
wait_for_instances(conn, master_nodes)
wait_for_instances(conn, slave_nodes)
if zoo_nodes != []:
wait_for_instances(conn, zoo_nodes)
print "Waiting %d more seconds..." % wait_secs
time.sleep(wait_secs)
@ -455,7 +456,12 @@ def get_num_disks(instance_type):
"m2.4xlarge": 2,
"cc1.4xlarge": 2,
"cc2.8xlarge": 4,
"cg1.4xlarge": 2
"cg1.4xlarge": 2,
"hs1.8xlarge": 24,
"cr1.8xlarge": 2,
"hi1.4xlarge": 2,
"m3.xlarge": 0,
"m3.2xlarge": 0
}
if instance_type in disks_by_instance:
return disks_by_instance[instance_type]
@ -470,8 +476,7 @@ def get_num_disks(instance_type):
# cluster (e.g. lists of masters and slaves). Files are only deployed to
# the first master instance in the cluster, and we expect the setup
# script to be run on that instance to copy them to other nodes.
def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes,
modules):
def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
active_master = master_nodes[0].public_dns_name
num_disks = get_num_disks(opts.instance_type)
@ -484,28 +489,30 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes,
mapred_local_dirs += ",/mnt%d/hadoop/mrlocal" % i
spark_local_dirs += ",/mnt%d/spark" % i
if zoo_nodes != []:
zoo_list = '\n'.join([i.public_dns_name for i in zoo_nodes])
cluster_url = "zoo://" + ",".join(
["%s:2181/mesos" % i.public_dns_name for i in zoo_nodes])
elif opts.cluster_type == "mesos":
zoo_list = "NONE"
cluster_url = "%s:5050" % active_master
elif opts.cluster_type == "standalone":
zoo_list = "NONE"
cluster_url = "%s:7077" % active_master
if "." in opts.spark_version:
# Pre-built spark & shark deploy
(spark_v, shark_v) = get_spark_shark_version(opts)
else:
# Spark-only custom deploy
spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version)
shark_v = ""
modules = filter(lambda x: x != "shark", modules)
template_vars = {
"master_list": '\n'.join([i.public_dns_name for i in master_nodes]),
"active_master": active_master,
"slave_list": '\n'.join([i.public_dns_name for i in slave_nodes]),
"zoo_list": zoo_list,
"cluster_url": cluster_url,
"hdfs_data_dirs": hdfs_data_dirs,
"mapred_local_dirs": mapred_local_dirs,
"spark_local_dirs": spark_local_dirs,
"swap": str(opts.swap),
"modules": '\n'.join(modules)
"modules": '\n'.join(modules),
"spark_version": spark_v,
"shark_version": shark_v,
"hadoop_major_version": opts.hadoop_major_version
}
# Create a temp directory in which we will place all the files to be
@ -555,7 +562,7 @@ def ssh(host, opts, command):
except subprocess.CalledProcessError as e:
if (tries > 2):
raise e
print "Error connecting to host {0}, sleeping 30".format(e)
print "Couldn't connect to host {0}, waiting 30 seconds".format(e)
time.sleep(30)
tries = tries + 1
@ -594,20 +601,20 @@ def main():
if action == "launch":
if opts.resume:
(master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(
(master_nodes, slave_nodes) = get_existing_cluster(
conn, opts, cluster_name)
else:
(master_nodes, slave_nodes, zoo_nodes) = launch_cluster(
(master_nodes, slave_nodes) = launch_cluster(
conn, opts, cluster_name)
wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes, zoo_nodes)
setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, True)
wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes)
setup_cluster(conn, master_nodes, slave_nodes, opts, True)
elif action == "destroy":
response = raw_input("Are you sure you want to destroy the cluster " +
cluster_name + "?\nALL DATA ON ALL NODES WILL BE LOST!!\n" +
"Destroy cluster " + cluster_name + " (y/N): ")
if response == "y":
(master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(
(master_nodes, slave_nodes) = get_existing_cluster(
conn, opts, cluster_name, die_on_error=False)
print "Terminating master..."
for inst in master_nodes:
@ -615,15 +622,11 @@ def main():
print "Terminating slaves..."
for inst in slave_nodes:
inst.terminate()
if zoo_nodes != []:
print "Terminating zoo..."
for inst in zoo_nodes:
inst.terminate()
# Delete security groups as well
if opts.delete_groups:
print "Deleting security groups (this will take some time)..."
group_names = [cluster_name + "-master", cluster_name + "-slaves", cluster_name + "-zoo"]
group_names = [cluster_name + "-master", cluster_name + "-slaves"]
attempt = 1;
while attempt <= 3:
@ -663,7 +666,7 @@ def main():
print "Try re-running in a few minutes."
elif action == "login":
(master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(
(master_nodes, slave_nodes) = get_existing_cluster(
conn, opts, cluster_name)
master = master_nodes[0].public_dns_name
print "Logging into master " + master + "..."
@ -674,7 +677,7 @@ def main():
(opts.identity_file, proxy_opt, opts.user, master), shell=True)
elif action == "get-master":
(master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(conn, opts, cluster_name)
(master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name)
print master_nodes[0].public_dns_name
elif action == "stop":
@ -684,7 +687,7 @@ def main():
"AMAZON EBS IF IT IS EBS-BACKED!!\n" +
"Stop cluster " + cluster_name + " (y/N): ")
if response == "y":
(master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(
(master_nodes, slave_nodes) = get_existing_cluster(
conn, opts, cluster_name, die_on_error=False)
print "Stopping master..."
for inst in master_nodes:
@ -694,15 +697,9 @@ def main():
for inst in slave_nodes:
if inst.state not in ["shutting-down", "terminated"]:
inst.stop()
if zoo_nodes != []:
print "Stopping zoo..."
for inst in zoo_nodes:
if inst.state not in ["shutting-down", "terminated"]:
inst.stop()
elif action == "start":
(master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(
conn, opts, cluster_name)
(master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name)
print "Starting slaves..."
for inst in slave_nodes:
if inst.state not in ["shutting-down", "terminated"]:
@ -711,13 +708,8 @@ def main():
for inst in master_nodes:
if inst.state not in ["shutting-down", "terminated"]:
inst.start()
if zoo_nodes != []:
print "Starting zoo..."
for inst in zoo_nodes:
if inst.state not in ["shutting-down", "terminated"]:
inst.start()
wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes, zoo_nodes)
setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, False)
wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes)
setup_cluster(conn, master_nodes, slave_nodes, opts, False)
else:
print >> stderr, "Invalid action: %s" % action

Some files were not shown because too many files have changed in this diff Show more