Merging upstream changes.
This commit is contained in:
commit
9afd0e2375
45
README.md
45
README.md
|
@ -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
|
||||
|
|
|
@ -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>
|
||||
|
|
105
bagel/pom.xml
105
bagel/pom.xml
|
@ -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>
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
200
core/pom.xml
200
core/pom.xml
|
@ -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>
|
||||
|
|
|
@ -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 }
|
||||
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,9 +248,8 @@ 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) =>
|
||||
for (v <- vs.iterator; w <- ws.iterator) yield (v, w)
|
||||
this.cogroup(other, partitioner).flatMapValues { case (vs, ws) =>
|
||||
for (v <- vs.iterator; w <- ws.iterator) yield (v, w)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -261,14 +259,15 @@ 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) =>
|
||||
if (ws.isEmpty) {
|
||||
vs.iterator.map(v => (v, None))
|
||||
} else {
|
||||
for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w))
|
||||
}
|
||||
|
||||
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 {
|
||||
for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -280,13 +279,12 @@ 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) =>
|
||||
if (vs.isEmpty) {
|
||||
ws.iterator.map(w => (None, w))
|
||||
} else {
|
||||
for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w)
|
||||
}
|
||||
this.cogroup(other, partitioner).flatMapValues { case (vs, ws) =>
|
||||
if (vs.isEmpty) {
|
||||
ws.iterator.map(w => (None, w))
|
||||
} else {
|
||||
for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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,13 +410,10 @@ 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) =>
|
||||
(vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
|
||||
prfs.mapValues { case Seq(vs, ws) =>
|
||||
(vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -425,15 +426,10 @@ 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) =>
|
||||
(vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]])
|
||||
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[_]]]
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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() {}
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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())
|
|
@ -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
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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_
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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) = {
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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(
|
||||
index: Int,
|
||||
@transient rdd: RDD[_],
|
||||
parentsIndices: Array[Int]
|
||||
) extends Partition {
|
||||
/**
|
||||
* 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],
|
||||
@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)
|
||||
@transient var prev: RDD[T],
|
||||
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
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
|
|
|
@ -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) =>
|
||||
|
|
|
@ -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)) }
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
51
core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala
Normal file
51
core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala
Normal 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)
|
||||
}
|
||||
}
|
|
@ -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))
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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],
|
||||
|
|
|
@ -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,23 +435,24 @@ class DAGScheduler(
|
|||
if (event != null) {
|
||||
logDebug("Got event of type " + event.getClass.getName)
|
||||
}
|
||||
|
||||
if (event != null) {
|
||||
if (processEvent(event)) {
|
||||
return
|
||||
this.synchronized { // needed in case other threads makes calls into methods of this class
|
||||
if (event != null) {
|
||||
if (processEvent(event)) {
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability
|
||||
// Periodically resubmit failed stages if some map output fetches have failed and we have
|
||||
// waited at least RESUBMIT_TIMEOUT. We wait for this short time because when a node fails,
|
||||
// tasks on many other nodes are bound to get a fetch failure, and they won't all get it at
|
||||
// the same time, so we want to make sure we've identified all the reduce tasks that depend
|
||||
// on the failed node.
|
||||
if (failed.size > 0 && time > lastFetchFailureTime + RESUBMIT_TIMEOUT) {
|
||||
resubmitFailedStages()
|
||||
} else {
|
||||
submitWaitingStages()
|
||||
val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability
|
||||
// Periodically resubmit failed stages if some map output fetches have failed and we have
|
||||
// waited at least RESUBMIT_TIMEOUT. We wait for this short time because when a node fails,
|
||||
// tasks on many other nodes are bound to get a fetch failure, and they won't all get it at
|
||||
// the same time, so we want to make sure we've identified all the reduce tasks that depend
|
||||
// on the failed node.
|
||||
if (failed.size > 0 && time > lastFetchFailureTime + RESUBMIT_TIMEOUT) {
|
||||
resubmitFailedStages()
|
||||
} else {
|
||||
submitWaitingStages()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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] {
|
||||
|
|
|
@ -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[_, _] =
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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]
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,172 +182,55 @@ 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 {
|
||||
SparkEnv.set(sc.env)
|
||||
// Mark each slave as alive and remember its hostname
|
||||
for (o <- offers) {
|
||||
// DEBUG Code
|
||||
Utils.checkHostPort(o.hostPort)
|
||||
def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized {
|
||||
SparkEnv.set(sc.env)
|
||||
|
||||
executorIdToHostPort(o.executorId) = o.hostPort
|
||||
if (! executorsByHostPort.contains(o.hostPort)) {
|
||||
executorsByHostPort(o.hostPort) = new HashSet[String]()
|
||||
}
|
||||
|
||||
hostPortsAlive += o.hostPort
|
||||
hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(o.hostPort)._1, new HashSet[String]).add(o.hostPort)
|
||||
executorGained(o.executorId, o.hostPort)
|
||||
// Mark each slave as alive and remember its hostname
|
||||
for (o <- offers) {
|
||||
executorIdToHost(o.executorId) = o.host
|
||||
if (!executorsByHost.contains(o.host)) {
|
||||
executorsByHost(o.host) = new HashSet[String]()
|
||||
executorGained(o.executorId, o.host)
|
||||
}
|
||||
// Build a list of tasks to assign to each slave
|
||||
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)
|
||||
}
|
||||
|
||||
map
|
||||
}
|
||||
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
|
||||
}
|
||||
|
||||
do {
|
||||
launchedTask = false
|
||||
var loopCount = 0
|
||||
for (i <- offersPriorityList) {
|
||||
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) =>
|
||||
tasks(i) += task
|
||||
val tid = task.taskId
|
||||
taskIdToTaskSetId(tid) = manager.taskSet.id
|
||||
taskSetTaskIds(manager.taskSet.id) += tid
|
||||
taskIdToExecutorId(tid) = execId
|
||||
activeExecutorIds += execId
|
||||
executorsByHostPort(hostPort) += 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)
|
||||
}
|
||||
|
||||
if (tasks.size > 0) {
|
||||
hasLaunchedTask = true
|
||||
}
|
||||
return tasks
|
||||
}
|
||||
|
||||
// Build a list of tasks to assign to each worker
|
||||
val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores))
|
||||
val availableCpus = offers.map(o => o.cores).toArray
|
||||
val sortedTaskSets = rootPool.getSortedTaskSetQueue()
|
||||
for (taskSet <- sortedTaskSets) {
|
||||
logDebug("parentName: %s, name: %s, runningTasks: %s".format(
|
||||
taskSet.parent.name, taskSet.name, taskSet.runningTasks))
|
||||
}
|
||||
|
||||
// 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
|
||||
for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) {
|
||||
do {
|
||||
launchedTask = false
|
||||
for (i <- 0 until offers.size) {
|
||||
val execId = offers(i).executorId
|
||||
val host = offers(i).host
|
||||
for (task <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) {
|
||||
tasks(i) += task
|
||||
val tid = task.taskId
|
||||
taskIdToTaskSetId(tid) = taskSet.taskSet.id
|
||||
taskSetTaskIds(taskSet.taskSet.id) += tid
|
||||
taskIdToExecutorId(tid) = execId
|
||||
activeExecutorIds += execId
|
||||
executorsByHost(host) += execId
|
||||
availableCpus(i) -= 1
|
||||
launchedTask = true
|
||||
}
|
||||
}
|
||||
} while (launchedTask)
|
||||
}
|
||||
|
||||
if (tasks.size > 0) {
|
||||
hasLaunchedTask = true
|
||||
}
|
||||
return tasks
|
||||
}
|
||||
|
||||
def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
|
||||
|
@ -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)
|
||||
}
|
||||
|
||||
None
|
||||
def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized {
|
||||
executorsByHost.get(host).map(_.toSet)
|
||||
}
|
||||
|
||||
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 hasExecutorsAliveOnHost(host: String): Boolean = synchronized {
|
||||
executorsByHost.contains(host)
|
||||
}
|
||||
|
||||
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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
|
||||
if (!readding) {
|
||||
allPendingTasks += index // No point scanning this whole list to find the old task there
|
||||
}
|
||||
}
|
||||
|
||||
// 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())
|
||||
/**
|
||||
* 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(hostPort: String): ArrayBuffer[Int] = {
|
||||
val host = Utils.parseHostPort(hostPort)._1
|
||||
/**
|
||||
* 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)
|
||||
}
|
||||
|
||||
if (rackTask != None) {
|
||||
speculatableTasks -= rackTask.get
|
||||
return rackTask
|
||||
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))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// 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.RACK_LOCAL)) {
|
||||
val rackLocalTask = findTaskFromList(getRackLocalPendingTasksForHost(hostPort))
|
||||
if (rackLocalTask != None) {
|
||||
return rackLocalTask
|
||||
if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) {
|
||||
for (index <- findTaskFromList(getPendingTasksForHost(host))) {
|
||||
return Some((index, TaskLocality.NODE_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
|
||||
if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) {
|
||||
for {
|
||||
rack <- sched.getRackForHost(host)
|
||||
index <- findTaskFromList(getPendingTasksForRack(rack))
|
||||
} {
|
||||
return Some((index, TaskLocality.RACK_LOCAL))
|
||||
}
|
||||
}
|
||||
|
||||
// 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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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) {
|
||||
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]]()
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
||||
executorLocations.put(blockId, executors.toSeq.toList)
|
||||
val blockLocations: Seq[Seq[BlockManagerId]] = if (env != null) {
|
||||
env.blockManager.getLocationBlockIds(blockIds)
|
||||
} else {
|
||||
blockManagerMaster.getLocations(blockIds)
|
||||
}
|
||||
|
||||
executorLocations
|
||||
val blockManagers = new HashMap[String, Seq[BlockManagerId]]
|
||||
for (i <- 0 until blockIds.length) {
|
||||
blockManagers(blockIds(i)) = blockLocations(i)
|
||||
}
|
||||
blockManagers.toMap
|
||||
}
|
||||
|
||||
def blockIdsToExecutorIds(
|
||||
blockIds: Array[String],
|
||||
env: SparkEnv,
|
||||
blockManagerMaster: BlockManagerMaster = null)
|
||||
: Map[String, Seq[String]] =
|
||||
{
|
||||
blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId))
|
||||
}
|
||||
|
||||
def blockIdsToHosts(
|
||||
blockIds: Array[String],
|
||||
env: SparkEnv,
|
||||
blockManagerMaster: BlockManagerMaster = null)
|
||||
: Map[String, Seq[String]] =
|
||||
{
|
||||
blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
@ -58,9 +58,9 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging {
|
|||
server = Some(srv)
|
||||
boundPort = Some(usedPort)
|
||||
} catch {
|
||||
case e: Exception =>
|
||||
logError("Failed to create Spark JettyUtils", e)
|
||||
System.exit(1)
|
||||
case e: Exception =>
|
||||
logError("Failed to create Spark JettyUtils", e)
|
||||
System.exit(1)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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"
|
||||
}
|
||||
|
|
|
@ -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}
|
||||
|
|
29
core/src/main/scala/spark/util/Clock.scala
Normal file
29
core/src/main/scala/spark/util/Clock.scala
Normal 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()
|
||||
}
|
36
core/src/main/scala/spark/util/MutablePair.scala
Normal file
36
core/src/main/scala/spark/util/MutablePair.scala
Normal 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[_,_]]
|
||||
}
|
|
@ -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)
|
||||
})
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)))
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
@ -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)))
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.scheduler
|
||||
package spark.scheduler.local
|
||||
|
||||
import org.scalatest.FunSuite
|
||||
import org.scalatest.BeforeAndAfter
|
|
@ -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") {
|
||||
|
|
26
core/src/test/scala/spark/util/FakeClock.scala
Normal file
26
core/src/test/scala/spark/util/FakeClock.scala
Normal 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
|
||||
}
|
|
@ -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"]
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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).
|
||||
|
|
|
@ -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 \
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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}}"
|
|
@ -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}}"
|
||||
|
|
248
ec2/spark_ec2.py
248
ec2/spark_ec2.py
|
@ -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))
|
||||
"group %s or %s" % (master_group.name, slave_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
|
||||
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")
|
||||
# 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 -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)
|
||||
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,23 +424,18 @@ 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
|
||||
print "Spark standalone cluster started at http://%s:8080" % master
|
||||
|
||||
if opts.ganglia:
|
||||
print "Ganglia started at http://%s:5080/ganglia" % master
|
||||
|
||||
|
||||
# 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
|
||||
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
Loading…
Reference in a new issue