diff --git a/.gitignore b/.gitignore
index 3b9086c7187dc..9757054a50f9e 100644
--- a/.gitignore
+++ b/.gitignore
@@ -8,16 +8,19 @@
*.pyc
.idea/
.idea_modules/
-sbt/*.jar
+build/*.jar
.settings
.cache
+cache
.generated-mima*
-/build/
work/
out/
.DS_Store
third_party/libmesos.so
third_party/libmesos.dylib
+build/apache-maven*
+build/zinc*
+build/scala*
conf/java-opts
conf/*.sh
conf/*.cmd
@@ -51,10 +54,11 @@ checkpoint
derby.log
dist/
dev/create-release/*txt
-dev/create-release/*new
+dev/create-release/*final
spark-*-bin-*.tgz
unit-tests.log
/lib/
+ec2/lib/
rat-results.txt
scalastyle.txt
scalastyle-output.xml
diff --git a/.rat-excludes b/.rat-excludes
index d8bee1f8e49c9..769defbac11b7 100644
--- a/.rat-excludes
+++ b/.rat-excludes
@@ -64,3 +64,4 @@ dist/*
logs
.*scalastyle-output.xml
.*dependency-reduced-pom.xml
+known_translations
diff --git a/README.md b/README.md
index 8d57d50da96c9..16628bd406775 100644
--- a/README.md
+++ b/README.md
@@ -26,7 +26,7 @@ To build Spark and its example programs, run:
(You do not need to do this if you downloaded a pre-built package.)
More detailed documentation is available from the project site, at
-["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-with-maven.html).
+["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-spark.html).
## Interactive Scala Shell
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 78fb908f9a9ef..b2a9d0780ee2b 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -354,5 +354,25 @@
+
+
+
+ hadoop-provided
+
+ provided
+
+
+
+ hive-provided
+
+ provided
+
+
+
+ parquet-provided
+
+ provided
+
+
diff --git a/bagel/pom.xml b/bagel/pom.xml
index 0327ffa402671..510e92640eff8 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -40,15 +40,6 @@
spark-core_${scala.binary.version}
${project.version}
-
- org.eclipse.jetty
- jetty-server
-
-
- org.scalatest
- scalatest_${scala.binary.version}
- test
-
org.scalacheck
scalacheck_${scala.binary.version}
@@ -58,11 +49,5 @@
target/scala-${scala.binary.version}/classes
target/scala-${scala.binary.version}/test-classes
-
-
- org.scalatest
- scalatest-maven-plugin
-
-
diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties
index 789869f72e3b0..853ef0ed2986f 100644
--- a/bagel/src/test/resources/log4j.properties
+++ b/bagel/src/test/resources/log4j.properties
@@ -15,10 +15,10 @@
# limitations under the License.
#
-# Set everything to be logged to the file bagel/target/unit-tests.log
+# Set everything to be logged to the file target/unit-tests.log
log4j.rootCategory=INFO, file
log4j.appender.file=org.apache.log4j.FileAppender
-log4j.appender.file.append=false
+log4j.appender.file.append=true
log4j.appender.file.file=target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd
index a4c099fb45b14..088f993954d9e 100644
--- a/bin/compute-classpath.cmd
+++ b/bin/compute-classpath.cmd
@@ -109,6 +109,13 @@ if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir
set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%
:no_yarn_conf_dir
+rem To allow for distributions to append needed libraries to the classpath (e.g. when
+rem using the "hadoop-provided" profile to build Spark), check SPARK_DIST_CLASSPATH and
+rem append it to tbe final classpath.
+if not "x%$SPARK_DIST_CLASSPATH%"=="x" (
+ set CLASSPATH=%CLASSPATH%;%SPARK_DIST_CLASSPATH%
+)
+
rem A bit of a hack to allow calling this script within run2.cmd without seeing output
if "%DONT_PRINT_CLASSPATH%"=="1" goto exit
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
index 685051eeed9f1..8f3b396ffd086 100755
--- a/bin/compute-classpath.sh
+++ b/bin/compute-classpath.sh
@@ -25,7 +25,11 @@ FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
. "$FWDIR"/bin/load-spark-env.sh
-CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH"
+if [ -n "$SPARK_CLASSPATH" ]; then
+ CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH"
+else
+ CLASSPATH="$SPARK_SUBMIT_CLASSPATH"
+fi
# Build up classpath
if [ -n "$SPARK_CONF_DIR" ]; then
@@ -142,4 +146,11 @@ if [ -n "$YARN_CONF_DIR" ]; then
CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
fi
+# To allow for distributions to append needed libraries to the classpath (e.g. when
+# using the "hadoop-provided" profile to build Spark), check SPARK_DIST_CLASSPATH and
+# append it to tbe final classpath.
+if [ -n "$SPARK_DIST_CLASSPATH" ]; then
+ CLASSPATH="$CLASSPATH:$SPARK_DIST_CLASSPATH"
+fi
+
echo "$CLASSPATH"
diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd
index 2ee60b4e2a2b3..1d1a40da315eb 100644
--- a/bin/spark-shell2.cmd
+++ b/bin/spark-shell2.cmd
@@ -19,4 +19,23 @@ rem
set SPARK_HOME=%~dp0..
-cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %* spark-shell
+echo "%*" | findstr " --help -h" >nul
+if %ERRORLEVEL% equ 0 (
+ call :usage
+ exit /b 0
+)
+
+call %SPARK_HOME%\bin\windows-utils.cmd %*
+if %ERRORLEVEL% equ 1 (
+ call :usage
+ exit /b 1
+)
+
+cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %SUBMISSION_OPTS% spark-shell %APPLICATION_OPTS%
+
+exit /b 0
+
+:usage
+echo "Usage: .\bin\spark-shell.cmd [options]" >&2
+%SPARK_HOME%\bin\spark-submit --help 2>&1 | findstr /V "Usage" 1>&2
+exit /b 0
diff --git a/bin/spark-submit b/bin/spark-submit
index f92d90c3a66b0..aefd38a0a2b90 100755
--- a/bin/spark-submit
+++ b/bin/spark-submit
@@ -38,11 +38,16 @@ while (($#)); do
export SPARK_SUBMIT_CLASSPATH=$2
elif [ "$1" = "--driver-java-options" ]; then
export SPARK_SUBMIT_OPTS=$2
+ elif [ "$1" = "--master" ]; then
+ export MASTER=$2
fi
shift
done
DEFAULT_PROPERTIES_FILE="$SPARK_HOME/conf/spark-defaults.conf"
+if [ "$MASTER" == "yarn-cluster" ]; then
+ SPARK_SUBMIT_DEPLOY_MODE=cluster
+fi
export SPARK_SUBMIT_DEPLOY_MODE=${SPARK_SUBMIT_DEPLOY_MODE:-"client"}
export SPARK_SUBMIT_PROPERTIES_FILE=${SPARK_SUBMIT_PROPERTIES_FILE:-"$DEFAULT_PROPERTIES_FILE"}
diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd
index cf6046d1547ad..daf0284db9230 100644
--- a/bin/spark-submit2.cmd
+++ b/bin/spark-submit2.cmd
@@ -45,11 +45,17 @@ if [%1] == [] goto continue
set SPARK_SUBMIT_CLASSPATH=%2
) else if [%1] == [--driver-java-options] (
set SPARK_SUBMIT_OPTS=%2
+ ) else if [%1] == [--master] (
+ set MASTER=%2
)
shift
goto loop
:continue
+if [%MASTER%] == [yarn-cluster] (
+ set SPARK_SUBMIT_DEPLOY_MODE=cluster
+)
+
rem For client mode, the driver will be launched in the same JVM that launches
rem SparkSubmit, so we may need to read the properties file for any extra class
rem paths, library paths, java options and memory early on. Otherwise, it will
diff --git a/bin/windows-utils.cmd b/bin/windows-utils.cmd
new file mode 100644
index 0000000000000..1082a952dac99
--- /dev/null
+++ b/bin/windows-utils.cmd
@@ -0,0 +1,59 @@
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements. See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License. You may obtain a copy of the License at
+rem
+rem http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
+rem Gather all spark-submit options into SUBMISSION_OPTS
+
+set SUBMISSION_OPTS=
+set APPLICATION_OPTS=
+
+rem NOTE: If you add or remove spark-sumbmit options,
+rem modify NOT ONLY this script but also SparkSubmitArgument.scala
+
+:OptsLoop
+if "x%1"=="x" (
+ goto :OptsLoopEnd
+)
+
+SET opts="\<--master\> \<--deploy-mode\> \<--class\> \<--name\> \<--jars\> \<--py-files\> \<--files\>"
+SET opts="%opts:~1,-1% \<--conf\> \<--properties-file\> \<--driver-memory\> \<--driver-java-options\>"
+SET opts="%opts:~1,-1% \<--driver-library-path\> \<--driver-class-path\> \<--executor-memory\>"
+SET opts="%opts:~1,-1% \<--driver-cores\> \<--total-executor-cores\> \<--executor-cores\> \<--queue\>"
+SET opts="%opts:~1,-1% \<--num-executors\> \<--archives\>"
+
+echo %1 | findstr %opts% >nul
+if %ERRORLEVEL% equ 0 (
+ if "x%2"=="x" (
+ echo "%1" requires an argument. >&2
+ exit /b 1
+ )
+ set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1 %2
+ shift
+ shift
+ goto :OptsLoop
+)
+echo %1 | findstr "\<--verbose\> \<-v\> \<--supervise\>" >nul
+if %ERRORLEVEL% equ 0 (
+ set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1
+ shift
+ goto :OptsLoop
+)
+set APPLICATION_OPTS=%APPLICATION_OPTS% %1
+shift
+goto :OptsLoop
+
+:OptsLoopEnd
+exit /b 0
diff --git a/build/mvn b/build/mvn
new file mode 100755
index 0000000000000..43471f83e904c
--- /dev/null
+++ b/build/mvn
@@ -0,0 +1,149 @@
+#!/usr/bin/env bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Determine the current working directory
+_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+# Preserve the calling directory
+_CALLING_DIR="$(pwd)"
+
+# Installs any application tarball given a URL, the expected tarball name,
+# and, optionally, a checkable binary path to determine if the binary has
+# already been installed
+## Arg1 - URL
+## Arg2 - Tarball Name
+## Arg3 - Checkable Binary
+install_app() {
+ local remote_tarball="$1/$2"
+ local local_tarball="${_DIR}/$2"
+ local binary="${_DIR}/$3"
+
+ # setup `curl` and `wget` silent options if we're running on Jenkins
+ local curl_opts=""
+ local wget_opts=""
+ if [ -n "$AMPLAB_JENKINS" ]; then
+ curl_opts="-s"
+ wget_opts="--quiet"
+ else
+ curl_opts="--progress-bar"
+ wget_opts="--progress=bar:force"
+ fi
+
+ if [ -z "$3" -o ! -f "$binary" ]; then
+ # check if we already have the tarball
+ # check if we have curl installed
+ # download application
+ [ ! -f "${local_tarball}" ] && [ -n "`which curl 2>/dev/null`" ] && \
+ echo "exec: curl ${curl_opts} ${remote_tarball}" && \
+ curl ${curl_opts} "${remote_tarball}" > "${local_tarball}"
+ # if the file still doesn't exist, lets try `wget` and cross our fingers
+ [ ! -f "${local_tarball}" ] && [ -n "`which wget 2>/dev/null`" ] && \
+ echo "exec: wget ${wget_opts} ${remote_tarball}" && \
+ wget ${wget_opts} -O "${local_tarball}" "${remote_tarball}"
+ # if both were unsuccessful, exit
+ [ ! -f "${local_tarball}" ] && \
+ echo -n "ERROR: Cannot download $2 with cURL or wget; " && \
+ echo "please install manually and try again." && \
+ exit 2
+ cd "${_DIR}" && tar -xzf "$2"
+ rm -rf "$local_tarball"
+ fi
+}
+
+# Install maven under the build/ folder
+install_mvn() {
+ install_app \
+ "http://apache.claz.org/maven/maven-3/3.2.3/binaries" \
+ "apache-maven-3.2.3-bin.tar.gz" \
+ "apache-maven-3.2.3/bin/mvn"
+ MVN_BIN="${_DIR}/apache-maven-3.2.3/bin/mvn"
+}
+
+# Install zinc under the build/ folder
+install_zinc() {
+ local zinc_path="zinc-0.3.5.3/bin/zinc"
+ [ ! -f "${zinc_path}" ] && ZINC_INSTALL_FLAG=1
+ install_app \
+ "http://downloads.typesafe.com/zinc/0.3.5.3" \
+ "zinc-0.3.5.3.tgz" \
+ "${zinc_path}"
+ ZINC_BIN="${_DIR}/${zinc_path}"
+}
+
+# Determine the Scala version from the root pom.xml file, set the Scala URL,
+# and, with that, download the specific version of Scala necessary under
+# the build/ folder
+install_scala() {
+ # determine the Scala version used in Spark
+ local scala_version=`grep "scala.version" "${_DIR}/../pom.xml" | \
+ head -1 | cut -f2 -d'>' | cut -f1 -d'<'`
+ local scala_bin="${_DIR}/scala-${scala_version}/bin/scala"
+
+ install_app \
+ "http://downloads.typesafe.com/scala/${scala_version}" \
+ "scala-${scala_version}.tgz" \
+ "scala-${scala_version}/bin/scala"
+
+ SCALA_COMPILER="$(cd "$(dirname ${scala_bin})/../lib" && pwd)/scala-compiler.jar"
+ SCALA_LIBRARY="$(cd "$(dirname ${scala_bin})/../lib" && pwd)/scala-library.jar"
+}
+
+# Determines if a given application is already installed. If not, will attempt
+# to install
+## Arg1 - application name
+## Arg2 - Alternate path to local install under build/ dir
+check_and_install_app() {
+ # create the local environment variable in uppercase
+ local app_bin="`echo $1 | awk '{print toupper(\$0)}'`_BIN"
+ # some black magic to set the generated app variable (i.e. MVN_BIN) into the
+ # environment
+ eval "${app_bin}=`which $1 2>/dev/null`"
+
+ if [ -z "`which $1 2>/dev/null`" ]; then
+ install_$1
+ fi
+}
+
+# Setup healthy defaults for the Zinc port if none were provided from
+# the environment
+ZINC_PORT=${ZINC_PORT:-"3030"}
+
+# Check and install all applications necessary to build Spark
+check_and_install_app "mvn"
+
+# Install the proper version of Scala and Zinc for the build
+install_zinc
+install_scala
+
+# Reset the current working directory
+cd "${_CALLING_DIR}"
+
+# Now that zinc is ensured to be installed, check its status and, if its
+# not running or just installed, start it
+if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`${ZINC_BIN} -status`" ]; then
+ ${ZINC_BIN} -shutdown
+ ${ZINC_BIN} -start -port ${ZINC_PORT} \
+ -scala-compiler "${SCALA_COMPILER}" \
+ -scala-library "${SCALA_LIBRARY}" &>/dev/null
+fi
+
+# Set any `mvn` options if not already present
+export MAVEN_OPTS=${MAVEN_OPTS:-"-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"}
+
+# Last, call the `mvn` command as usual
+${MVN_BIN} "$@"
diff --git a/build/sbt b/build/sbt
new file mode 100755
index 0000000000000..28ebb64f7197c
--- /dev/null
+++ b/build/sbt
@@ -0,0 +1,128 @@
+#!/usr/bin/env bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# When creating new tests for Spark SQL Hive, the HADOOP_CLASSPATH must contain the hive jars so
+# that we can run Hive to generate the golden answer. This is not required for normal development
+# or testing.
+for i in "$HIVE_HOME"/lib/*
+do HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$i"
+done
+export HADOOP_CLASSPATH
+
+realpath () {
+(
+ TARGET_FILE="$1"
+
+ cd "$(dirname "$TARGET_FILE")"
+ TARGET_FILE="$(basename "$TARGET_FILE")"
+
+ COUNT=0
+ while [ -L "$TARGET_FILE" -a $COUNT -lt 100 ]
+ do
+ TARGET_FILE="$(readlink "$TARGET_FILE")"
+ cd $(dirname "$TARGET_FILE")
+ TARGET_FILE="$(basename $TARGET_FILE)"
+ COUNT=$(($COUNT + 1))
+ done
+
+ echo "$(pwd -P)/"$TARGET_FILE""
+)
+}
+
+. "$(dirname "$(realpath "$0")")"/sbt-launch-lib.bash
+
+
+declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy"
+declare -r sbt_opts_file=".sbtopts"
+declare -r etc_sbt_opts_file="/etc/sbt/sbtopts"
+
+usage() {
+ cat < path to global settings/plugins directory (default: ~/.sbt)
+ -sbt-boot path to shared boot directory (default: ~/.sbt/boot in 0.11 series)
+ -ivy path to local Ivy repository (default: ~/.ivy2)
+ -mem set memory options (default: $sbt_mem, which is $(get_mem_opts $sbt_mem))
+ -no-share use all local caches; no sharing
+ -no-global uses global caches, but does not use global ~/.sbt directory.
+ -jvm-debug Turn on JVM debugging, open at the given port.
+ -batch Disable interactive mode
+
+ # sbt version (default: from project/build.properties if present, else latest release)
+ -sbt-version use the specified version of sbt
+ -sbt-jar use the specified jar as the sbt launcher
+ -sbt-rc use an RC version of sbt
+ -sbt-snapshot use a snapshot version of sbt
+
+ # java version (default: java from PATH, currently $(java -version 2>&1 | grep version))
+ -java-home alternate JAVA_HOME
+
+ # jvm options and output control
+ JAVA_OPTS environment variable, if unset uses "$java_opts"
+ SBT_OPTS environment variable, if unset uses "$default_sbt_opts"
+ .sbtopts if this file exists in the current directory, it is
+ prepended to the runner args
+ /etc/sbt/sbtopts if this file exists, it is prepended to the runner args
+ -Dkey=val pass -Dkey=val directly to the java runtime
+ -J-X pass option -X directly to the java runtime
+ (-J is stripped)
+ -S-X add -X to sbt's scalacOptions (-S is stripped)
+ -PmavenProfiles Enable a maven profile for the build.
+
+In the case of duplicated or conflicting options, the order above
+shows precedence: JAVA_OPTS lowest, command line options highest.
+EOM
+}
+
+process_my_args () {
+ while [[ $# -gt 0 ]]; do
+ case "$1" in
+ -no-colors) addJava "-Dsbt.log.noformat=true" && shift ;;
+ -no-share) addJava "$noshare_opts" && shift ;;
+ -no-global) addJava "-Dsbt.global.base=$(pwd)/project/.sbtboot" && shift ;;
+ -sbt-boot) require_arg path "$1" "$2" && addJava "-Dsbt.boot.directory=$2" && shift 2 ;;
+ -sbt-dir) require_arg path "$1" "$2" && addJava "-Dsbt.global.base=$2" && shift 2 ;;
+ -debug-inc) addJava "-Dxsbt.inc.debug=true" && shift ;;
+ -batch) exec selenium-java
test
-
- org.scalatest
- scalatest_${scala.binary.version}
- test
-
org.mockito
mockito-all
@@ -326,19 +321,6 @@
target/scala-${scala.binary.version}/classes
target/scala-${scala.binary.version}/test-classes
-
- org.scalatest
- scalatest-maven-plugin
-
-
- test
-
- test
-
-
-
-
-
org.apache.maven.plugins
@@ -352,9 +334,9 @@
-
+
-
+
diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css
index 68c52ac09b55a..5751964b792ce 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/webui.css
+++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css
@@ -169,6 +169,16 @@ span.additional-metric-title {
display: inline-block;
}
+.version {
+ line-height: 30px;
+ vertical-align: bottom;
+ font-size: 12px;
+ padding: 0;
+ margin: 0;
+ font-weight: bold;
+ color: #777;
+}
+
/* Hide all additional metrics by default. This is done here rather than using JavaScript to
* avoid slow page loads for stage pages with large numbers (e.g., thousands) of tasks. */
.scheduler_delay, .deserialization_time, .serialization_time, .getting_result_time {
diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala
index ab2594cfc02eb..9a7cd4523e5ab 100644
--- a/core/src/main/scala/org/apache/spark/Dependency.scala
+++ b/core/src/main/scala/org/apache/spark/Dependency.scala
@@ -60,6 +60,9 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] {
* @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None,
* the default serializer, as specified by `spark.serializer` config option, will
* be used.
+ * @param keyOrdering key ordering for RDD's shuffles
+ * @param aggregator map/reduce-side aggregator for RDD's shuffle
+ * @param mapSideCombine whether to perform partial aggregation (also known as map-side combine)
*/
@DeveloperApi
class ShuffleDependency[K, V, C](
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
new file mode 100644
index 0000000000000..a46a81eabd965
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+/**
+ * A client that communicates with the cluster manager to request or kill executors.
+ */
+private[spark] trait ExecutorAllocationClient {
+
+ /**
+ * Request an additional number of executors from the cluster manager.
+ * Return whether the request is acknowledged by the cluster manager.
+ */
+ def requestExecutors(numAdditionalExecutors: Int): Boolean
+
+ /**
+ * Request that the cluster manager kill the specified executors.
+ * Return whether the request is acknowledged by the cluster manager.
+ */
+ def killExecutors(executorIds: Seq[String]): Boolean
+
+ /**
+ * Request that the cluster manager kill the specified executor.
+ * Return whether the request is acknowledged by the cluster manager.
+ */
+ def killExecutor(executorId: String): Boolean = killExecutors(Seq(executorId))
+}
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index 88adb892998af..e9e90e3f2f65a 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -60,11 +60,13 @@ import org.apache.spark.scheduler._
* spark.dynamicAllocation.executorIdleTimeout (K) -
* If an executor has been idle for this duration, remove it
*/
-private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
+private[spark] class ExecutorAllocationManager(
+ client: ExecutorAllocationClient,
+ listenerBus: LiveListenerBus,
+ conf: SparkConf)
+ extends Logging {
import ExecutorAllocationManager._
- private val conf = sc.conf
-
// Lower and upper bounds on the number of executors. These are required.
private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
@@ -168,7 +170,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
* Register for scheduler callbacks to decide when to add and remove executors.
*/
def start(): Unit = {
- sc.addSparkListener(listener)
+ listenerBus.addListener(listener)
startPolling()
}
@@ -253,7 +255,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
val actualNumExecutorsToAdd = math.min(numExecutorsToAdd, maxNumExecutorsToAdd)
val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
- val addRequestAcknowledged = testing || sc.requestExecutors(actualNumExecutorsToAdd)
+ val addRequestAcknowledged = testing || client.requestExecutors(actualNumExecutorsToAdd)
if (addRequestAcknowledged) {
logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " +
s"tasks are backlogged (new desired total will be $newTotalExecutors)")
@@ -295,7 +297,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
}
// Send a request to the backend to kill this executor
- val removeRequestAcknowledged = testing || sc.killExecutor(executorId)
+ val removeRequestAcknowledged = testing || client.killExecutor(executorId)
if (removeRequestAcknowledged) {
logInfo(s"Removing executor $executorId because it has been idle for " +
s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})")
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 7d96962c4acd7..6e4edc7c80d7a 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -72,20 +72,22 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster
/**
* Class that keeps track of the location of the map output of
* a stage. This is abstract because different versions of MapOutputTracker
- * (driver and worker) use different HashMap to store its metadata.
+ * (driver and executor) use different HashMap to store its metadata.
*/
private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging {
private val timeout = AkkaUtils.askTimeout(conf)
+ private val retryAttempts = AkkaUtils.numRetries(conf)
+ private val retryIntervalMs = AkkaUtils.retryWaitMs(conf)
/** Set to the MapOutputTrackerActor living on the driver. */
var trackerActor: ActorRef = _
/**
- * This HashMap has different behavior for the master and the workers.
+ * This HashMap has different behavior for the driver and the executors.
*
- * On the master, it serves as the source of map outputs recorded from ShuffleMapTasks.
- * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the
- * master's corresponding HashMap.
+ * On the driver, it serves as the source of map outputs recorded from ShuffleMapTasks.
+ * On the executors, it simply serves as a cache, in which a miss triggers a fetch from the
+ * driver's corresponding HashMap.
*
* Note: because mapStatuses is accessed concurrently, subclasses should make sure it's a
* thread-safe map.
@@ -99,7 +101,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
protected var epoch: Long = 0
protected val epochLock = new AnyRef
- /** Remembers which map output locations are currently being fetched on a worker. */
+ /** Remembers which map output locations are currently being fetched on an executor. */
private val fetching = new HashSet[Int]
/**
@@ -108,8 +110,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
*/
protected def askTracker(message: Any): Any = {
try {
- val future = trackerActor.ask(message)(timeout)
- Await.result(future, timeout)
+ AkkaUtils.askWithReply(message, trackerActor, retryAttempts, retryIntervalMs, timeout)
} catch {
case e: Exception =>
logError("Error communicating with MapOutputTracker", e)
@@ -136,14 +137,12 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them")
var fetchedStatuses: Array[MapStatus] = null
fetching.synchronized {
- if (fetching.contains(shuffleId)) {
- // Someone else is fetching it; wait for them to be done
- while (fetching.contains(shuffleId)) {
- try {
- fetching.wait()
- } catch {
- case e: InterruptedException =>
- }
+ // Someone else is fetching it; wait for them to be done
+ while (fetching.contains(shuffleId)) {
+ try {
+ fetching.wait()
+ } catch {
+ case e: InterruptedException =>
}
}
@@ -198,8 +197,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
/**
* Called from executors 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.
+ * because of a fetch failure. Each executor task calls this with the latest epoch
+ * number on the driver at the time it was created.
*/
def updateEpoch(newEpoch: Long) {
epochLock.synchronized {
@@ -231,7 +230,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
private var cacheEpoch = epoch
/**
- * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the master,
+ * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the driver,
* so that statuses are dropped only by explicit de-registering or by TTL-based cleaning (if set).
* Other than these two scenarios, nothing should be dropped from this HashMap.
*/
@@ -341,7 +340,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
}
/**
- * MapOutputTracker for the workers, which fetches map output information from the driver's
+ * MapOutputTracker for the executors, which fetches map output information from the driver's
* MapOutputTrackerMaster.
*/
private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) {
diff --git a/core/src/main/scala/org/apache/spark/Partition.scala b/core/src/main/scala/org/apache/spark/Partition.scala
index 27892dbd2a0bc..dd3f28e4197e3 100644
--- a/core/src/main/scala/org/apache/spark/Partition.scala
+++ b/core/src/main/scala/org/apache/spark/Partition.scala
@@ -18,11 +18,11 @@
package org.apache.spark
/**
- * A partition of an RDD.
+ * An identifier for a partition in an RDD.
*/
trait Partition extends Serializable {
/**
- * Get the split's index within its parent RDD
+ * Get the partition's index within its parent RDD
*/
def index: Int
diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala
index dbff9d12b5ad7..ec82d09cd079b 100644
--- a/core/src/main/scala/org/apache/spark/SecurityManager.scala
+++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala
@@ -93,19 +93,19 @@ import org.apache.spark.network.sasl.SecretKeyHolder
* Note that SASL is pluggable as to what mechanism it uses. We currently use
* DIGEST-MD5 but this could be changed to use Kerberos or other in the future.
* Spark currently supports "auth" for the quality of protection, which means
- * the connection is not supporting integrity or privacy protection (encryption)
+ * the connection does not support integrity or privacy protection (encryption)
* after authentication. SASL also supports "auth-int" and "auth-conf" which
- * SPARK could be support in the future to allow the user to specify the quality
+ * SPARK could support in the future to allow the user to specify the quality
* of protection they want. If we support those, the messages will also have to
* be wrapped and unwrapped via the SaslServer/SaslClient.wrap/unwrap API's.
*
* Since the NioBlockTransferService does asynchronous messages passing, the SASL
* authentication is a bit more complex. A ConnectionManager can be both a client
- * and a Server, so for a particular connection is has to determine what to do.
+ * and a Server, so for a particular connection it has to determine what to do.
* A ConnectionId was added to be able to track connections and is used to
* match up incoming messages with connections waiting for authentication.
- * The ConnectionManager tracks all the sendingConnections using the ConnectionId
- * and waits for the response from the server and does the handshake before sending
+ * The ConnectionManager tracks all the sendingConnections using the ConnectionId,
+ * waits for the response from the server, and does the handshake before sending
* the real message.
*
* The NettyBlockTransferService ensures that SASL authentication is performed
@@ -114,14 +114,14 @@ import org.apache.spark.network.sasl.SecretKeyHolder
*
* - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters
* can be used. Yarn requires a specific AmIpFilter be installed for security to work
- * properly. For non-Yarn deployments, users can write a filter to go through a
- * companies normal login service. If an authentication filter is in place then the
+ * properly. For non-Yarn deployments, users can write a filter to go through their
+ * organization's normal login service. If an authentication filter is in place then the
* SparkUI can be configured to check the logged in user against the list of users who
* have view acls to see if that user is authorized.
* The filters can also be used for many different purposes. For instance filters
* could be used for logging, encryption, or compression.
*
- * The exact mechanisms used to generate/distributed the shared secret is deployment specific.
+ * The exact mechanisms used to generate/distribute the shared secret are deployment-specific.
*
* For Yarn deployments, the secret is automatically generated using the Akka remote
* Crypt.generateSecureCookie() API. The secret is placed in the Hadoop UGI which gets passed
@@ -138,7 +138,7 @@ import org.apache.spark.network.sasl.SecretKeyHolder
* All the nodes (Master and Workers) and the applications need to have the same shared secret.
* This again is not ideal as one user could potentially affect another users application.
* This should be enhanced in the future to provide better protection.
- * If the UI needs to be secured the user needs to install a javax servlet filter to do the
+ * If the UI needs to be secure, the user needs to install a javax servlet filter to do the
* authentication. Spark will then use that user to compare against the view acls to do
* authorization. If not filter is in place the user is generally null and no authorization
* can take place.
@@ -151,8 +151,8 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with
private val authOn = sparkConf.getBoolean("spark.authenticate", false)
// keep spark.ui.acls.enable for backwards compatibility with 1.0
- private var aclsOn = sparkConf.getOption("spark.acls.enable").getOrElse(
- sparkConf.get("spark.ui.acls.enable", "false")).toBoolean
+ private var aclsOn =
+ sparkConf.getBoolean("spark.acls.enable", sparkConf.getBoolean("spark.ui.acls.enable", false))
// admin acls should be set before view or modify acls
private var adminAcls: Set[String] =
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 8e5378ecc08de..3bf3acd245d8f 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -64,7 +64,7 @@ import org.apache.spark.util._
* @param config a Spark Config object describing the application configuration. Any settings in
* this config overrides the default configs as well as system properties.
*/
-class SparkContext(config: SparkConf) extends Logging {
+class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationClient {
// The call site where this SparkContext was constructed.
private val creationSite: CallSite = Utils.getCallSite()
@@ -172,6 +172,9 @@ class SparkContext(config: SparkConf) extends Logging {
private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) =
this(master, appName, sparkHome, jars, Map(), Map())
+ // log out Spark Version in Spark driver log
+ logInfo(s"Running Spark version $SPARK_VERSION")
+
private[spark] val conf = config.clone()
conf.validateSettings()
@@ -226,7 +229,7 @@ class SparkContext(config: SparkConf) extends Logging {
// An asynchronous listener bus for Spark events
private[spark] val listenerBus = new LiveListenerBus
- conf.set("spark.executor.id", "driver")
+ conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER)
// Create the Spark execution environment (cache, map output tracker, etc)
private[spark] val env = SparkEnv.createDriverEnv(conf, isLocal, listenerBus)
@@ -326,8 +329,13 @@ class SparkContext(config: SparkConf) extends Logging {
try {
dagScheduler = new DAGScheduler(this)
} catch {
- case e: Exception => throw
- new SparkException("DAGScheduler cannot be initialized due to %s".format(e.getMessage))
+ case e: Exception => {
+ try {
+ stop()
+ } finally {
+ throw new SparkException("Error while constructing DAGScheduler", e)
+ }
+ }
}
// start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's
@@ -344,6 +352,8 @@ class SparkContext(config: SparkConf) extends Logging {
// The metrics system for Driver need to be set spark.app.id to app ID.
// So it should start after we get app ID from the task scheduler and set spark.app.id.
metricsSystem.start()
+ // Attach the driver metrics servlet handler to the web ui after the metrics system is started.
+ metricsSystem.getServletHandlers.foreach(handler => ui.foreach(_.attachHandler(handler)))
// Optionally log Spark events
private[spark] val eventLogger: Option[EventLoggingListener] = {
@@ -363,7 +373,7 @@ class SparkContext(config: SparkConf) extends Logging {
if (dynamicAllocationEnabled) {
assert(master.contains("yarn") || dynamicAllocationTesting,
"Dynamic allocation of executors is currently only supported in YARN mode")
- Some(new ExecutorAllocationManager(this))
+ Some(new ExecutorAllocationManager(this, listenerBus, conf))
} else {
None
}
@@ -992,7 +1002,7 @@ class SparkContext(config: SparkConf) extends Logging {
* This is currently only supported in Yarn mode. Return whether the request is received.
*/
@DeveloperApi
- def requestExecutors(numAdditionalExecutors: Int): Boolean = {
+ override def requestExecutors(numAdditionalExecutors: Int): Boolean = {
assert(master.contains("yarn") || dynamicAllocationTesting,
"Requesting executors is currently only supported in YARN mode")
schedulerBackend match {
@@ -1010,7 +1020,7 @@ class SparkContext(config: SparkConf) extends Logging {
* This is currently only supported in Yarn mode. Return whether the request is received.
*/
@DeveloperApi
- def killExecutors(executorIds: Seq[String]): Boolean = {
+ override def killExecutors(executorIds: Seq[String]): Boolean = {
assert(master.contains("yarn") || dynamicAllocationTesting,
"Killing executors is currently only supported in YARN mode")
schedulerBackend match {
@@ -1028,7 +1038,7 @@ class SparkContext(config: SparkConf) extends Logging {
* This is currently only supported in Yarn mode. Return whether the request is received.
*/
@DeveloperApi
- def killExecutor(executorId: String): Boolean = killExecutors(Seq(executorId))
+ override def killExecutor(executorId: String): Boolean = super.killExecutor(executorId)
/** The version of Spark on which this application is running. */
def version = SPARK_VERSION
@@ -1703,19 +1713,19 @@ object SparkContext extends Logging {
// Implicit conversions to common Writable types, for saveAsSequenceFile
- implicit def intToIntWritable(i: Int) = new IntWritable(i)
+ implicit def intToIntWritable(i: Int): IntWritable = new IntWritable(i)
- implicit def longToLongWritable(l: Long) = new LongWritable(l)
+ implicit def longToLongWritable(l: Long): LongWritable = new LongWritable(l)
- implicit def floatToFloatWritable(f: Float) = new FloatWritable(f)
+ implicit def floatToFloatWritable(f: Float): FloatWritable = new FloatWritable(f)
- implicit def doubleToDoubleWritable(d: Double) = new DoubleWritable(d)
+ implicit def doubleToDoubleWritable(d: Double): DoubleWritable = new DoubleWritable(d)
- implicit def boolToBoolWritable (b: Boolean) = new BooleanWritable(b)
+ implicit def boolToBoolWritable (b: Boolean): BooleanWritable = new BooleanWritable(b)
- implicit def bytesToBytesWritable (aob: Array[Byte]) = new BytesWritable(aob)
+ implicit def bytesToBytesWritable (aob: Array[Byte]): BytesWritable = new BytesWritable(aob)
- implicit def stringToText(s: String) = new Text(s)
+ implicit def stringToText(s: String): Text = new Text(s)
private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T])
: ArrayWritable = {
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index e464b32e61dd6..43436a1697000 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -156,7 +156,15 @@ object SparkEnv extends Logging {
assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!")
val hostname = conf.get("spark.driver.host")
val port = conf.get("spark.driver.port").toInt
- create(conf, SparkContext.DRIVER_IDENTIFIER, hostname, port, true, isLocal, listenerBus)
+ create(
+ conf,
+ SparkContext.DRIVER_IDENTIFIER,
+ hostname,
+ port,
+ isDriver = true,
+ isLocal = isLocal,
+ listenerBus = listenerBus
+ )
}
/**
@@ -169,10 +177,18 @@ object SparkEnv extends Logging {
hostname: String,
port: Int,
numCores: Int,
- isLocal: Boolean,
- actorSystem: ActorSystem = null): SparkEnv = {
- create(conf, executorId, hostname, port, false, isLocal, defaultActorSystem = actorSystem,
- numUsableCores = numCores)
+ isLocal: Boolean): SparkEnv = {
+ val env = create(
+ conf,
+ executorId,
+ hostname,
+ port,
+ isDriver = false,
+ isLocal = isLocal,
+ numUsableCores = numCores
+ )
+ SparkEnv.set(env)
+ env
}
/**
@@ -186,7 +202,6 @@ object SparkEnv extends Logging {
isDriver: Boolean,
isLocal: Boolean,
listenerBus: LiveListenerBus = null,
- defaultActorSystem: ActorSystem = null,
numUsableCores: Int = 0): SparkEnv = {
// Listener bus is only used on the driver
@@ -196,20 +211,17 @@ object SparkEnv extends Logging {
val securityManager = new SecurityManager(conf)
- // If an existing actor system is already provided, use it.
- // This is the case when an executor is launched in coarse-grained mode.
- val (actorSystem, boundPort) =
- Option(defaultActorSystem) match {
- case Some(as) => (as, port)
- case None =>
- val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName
- AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager)
- }
+ // Create the ActorSystem for Akka and get the port it binds to.
+ val (actorSystem, boundPort) = {
+ val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName
+ AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager)
+ }
// Figure out which port Akka actually bound to in case the original port is 0 or occupied.
- // This is so that we tell the executors the correct port to connect to.
if (isDriver) {
conf.set("spark.driver.port", boundPort.toString)
+ } else {
+ conf.set("spark.executor.port", boundPort.toString)
}
// Create an instance of the class with the given name, possibly initializing it with our conf
@@ -383,7 +395,7 @@ object SparkEnv extends Logging {
val sparkProperties = (conf.getAll ++ schedulerMode).sorted
// System properties that are not java classpaths
- val systemProperties = System.getProperties.iterator.toSeq
+ val systemProperties = Utils.getSystemProperties.toSeq
val otherProperties = systemProperties.filter { case (k, _) =>
k != "java.class.path" && !k.startsWith("spark.")
}.sorted
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala
index 86e94931300f8..71b26737b8c02 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala
@@ -80,7 +80,7 @@ private[spark] object JavaUtils {
prev match {
case Some(k) =>
underlying match {
- case mm: mutable.Map[a, _] =>
+ case mm: mutable.Map[A, _] =>
mm remove k
prev = None
case _ =>
diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala
index f2687ce6b42b4..7c1c831c248fc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala
@@ -160,6 +160,8 @@ object Client {
val (actorSystem, _) = AkkaUtils.createActorSystem(
"driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf))
+ // Verify driverArgs.master is a valid url so that we can use it in ClientActor safely
+ Master.toAkkaUrl(driverArgs.master)
actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf))
actorSystem.awaitTermination()
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index c46f84de8444a..243d8edb72ed3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -88,6 +88,8 @@ private[deploy] object DeployMessages {
case class KillDriver(driverId: String) extends DeployMessage
+ case class ApplicationFinished(id: String)
+
// Worker internal
case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders
@@ -175,4 +177,5 @@ private[deploy] object DeployMessages {
// Liveness checks in various places
case object SendHeartbeat
+
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 60ee115e393ce..57f9faf5ddd1d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.fs.FileSystem.Statistics
import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext}
import org.apache.hadoop.security.Credentials
import org.apache.hadoop.security.UserGroupInformation
@@ -183,6 +184,17 @@ class SparkHadoopUtil extends Logging {
Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData")
statisticsDataClass.getDeclaredMethod(methodName)
}
+
+ /**
+ * Using reflection to get the Configuration from JobContext/TaskAttemptContext. If we directly
+ * call `JobContext/TaskAttemptContext.getConfiguration`, it will generate different byte codes
+ * for Hadoop 1.+ and Hadoop 2.+ because JobContext/TaskAttemptContext is class in Hadoop 1.+
+ * while it's interface in Hadoop 2.+.
+ */
+ def getConfigurationFromJobContext(context: JobContext): Configuration = {
+ val method = context.getClass.getMethod("getConfiguration")
+ method.invoke(context).asInstanceOf[Configuration]
+ }
}
object SparkHadoopUtil {
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index f0e9ee67f6a67..1faabe91f49a8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -17,6 +17,7 @@
package org.apache.spark.deploy
+import java.net.URI
import java.util.jar.JarFile
import scala.collection.mutable.{ArrayBuffer, HashMap}
@@ -120,17 +121,28 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull
jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull
deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull
+ numExecutors = Option(numExecutors)
+ .getOrElse(sparkProperties.get("spark.executor.instances").orNull)
// Try to set main class from JAR if no --class argument is given
if (mainClass == null && !isPython && primaryResource != null) {
- try {
- val jar = new JarFile(primaryResource)
- // Note that this might still return null if no main-class is set; we catch that later
- mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class")
- } catch {
- case e: Exception =>
- SparkSubmit.printErrorAndExit("Cannot load main class from JAR: " + primaryResource)
- return
+ val uri = new URI(primaryResource)
+ val uriScheme = uri.getScheme()
+
+ uriScheme match {
+ case "file" =>
+ try {
+ val jar = new JarFile(uri.getPath)
+ // Note that this might still return null if no main-class is set; we catch that later
+ mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class")
+ } catch {
+ case e: Exception =>
+ SparkSubmit.printErrorAndExit(s"Cannot load main class from JAR $primaryResource")
+ }
+ case _ =>
+ SparkSubmit.printErrorAndExit(
+ s"Cannot load main class from JAR $primaryResource with URI $uriScheme. " +
+ "Please specify a class through --class.")
}
}
@@ -212,7 +224,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
""".stripMargin
}
- /** Fill in values by parsing user options. */
+ /**
+ * Fill in values by parsing user options.
+ * NOTE: Any changes here must be reflected in YarnClientSchedulerBackend.
+ */
private def parseOpts(opts: Seq[String]): Unit = {
val EQ_SEPARATED_OPT="""(--[^=]+)=(.+)""".r
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala
index d2687faad62b1..2eab9981845e8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala
@@ -151,7 +151,8 @@ private[spark] object SparkSubmitDriverBootstrapper {
val isWindows = Utils.isWindows
val isSubprocess = sys.env.contains("IS_SUBPROCESS")
if (!isWindows) {
- val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin")
+ val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin",
+ propagateEof = true)
stdinThread.start()
// Spark submit (JVM) may run as a subprocess, and so this JVM should terminate on
// broken pipe, signaling that the parent process has exited. This is the case if the
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
index 4efebcaa350fe..39a7b0319b6a1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
@@ -26,7 +26,7 @@ import akka.actor._
import akka.pattern.ask
import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent}
-import org.apache.spark.{Logging, SparkConf, SparkException}
+import org.apache.spark.{Logging, SparkConf}
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.Master
@@ -47,6 +47,8 @@ private[spark] class AppClient(
conf: SparkConf)
extends Logging {
+ val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl)
+
val REGISTRATION_TIMEOUT = 20.seconds
val REGISTRATION_RETRIES = 3
@@ -75,9 +77,9 @@ private[spark] class AppClient(
}
def tryRegisterAllMasters() {
- for (masterUrl <- masterUrls) {
- logInfo("Connecting to master " + masterUrl + "...")
- val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
+ for (masterAkkaUrl <- masterAkkaUrls) {
+ logInfo("Connecting to master " + masterAkkaUrl + "...")
+ val actor = context.actorSelection(masterAkkaUrl)
actor ! RegisterApplication(appDescription)
}
}
@@ -103,20 +105,14 @@ private[spark] class AppClient(
}
def changeMaster(url: String) {
+ // activeMasterUrl is a valid Spark url since we receive it from master.
activeMasterUrl = url
master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
- masterAddress = activeMasterUrl match {
- case Master.sparkUrlRegex(host, port) =>
- Address("akka.tcp", Master.systemName, host, port.toInt)
- case x =>
- throw new SparkException("Invalid spark URL: " + x)
- }
+ masterAddress = Master.toAkkaAddress(activeMasterUrl)
}
private def isPossibleMaster(remoteUrl: Address) = {
- masterUrls.map(s => Master.toAkkaUrl(s))
- .map(u => AddressFromURIString(u).hostPort)
- .contains(remoteUrl.hostPort)
+ masterAkkaUrls.map(AddressFromURIString(_).hostPort).contains(remoteUrl.hostPort)
}
override def receiveWithLogging = {
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
index fbe39b27649f6..553bf3cb945ab 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
@@ -25,7 +25,8 @@ private[spark] case class ApplicationHistoryInfo(
startTime: Long,
endTime: Long,
lastUpdated: Long,
- sparkUser: String)
+ sparkUser: String,
+ completed: Boolean = false)
private[spark] abstract class ApplicationHistoryProvider {
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index 82a54dbfb5330..2b084a2d73b78 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -17,14 +17,16 @@
package org.apache.spark.deploy.history
-import java.io.FileNotFoundException
+import java.io.{BufferedInputStream, FileNotFoundException, InputStream}
import scala.collection.mutable
import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.hadoop.fs.permission.AccessControlException
import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.io.CompressionCodec
import org.apache.spark.scheduler._
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.Utils
@@ -64,6 +66,12 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
@volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo]
= new mutable.LinkedHashMap()
+ // Constants used to parse Spark 1.0.0 log directories.
+ private[history] val LOG_PREFIX = "EVENT_LOG_"
+ private[history] val SPARK_VERSION_PREFIX = "SPARK_VERSION_"
+ private[history] val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_"
+ private[history] val APPLICATION_COMPLETE = "APPLICATION_COMPLETE"
+
/**
* A background thread that periodically checks for event log updates on disk.
*
@@ -90,7 +98,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
initialize()
- private def initialize() {
+ private def initialize(): Unit = {
// Validate the log directory.
val path = new Path(logDir)
if (!fs.exists(path)) {
@@ -106,8 +114,12 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
}
checkForLogs()
- logCheckingThread.setDaemon(true)
- logCheckingThread.start()
+
+ // Disable the background thread during tests.
+ if (!conf.contains("spark.testing")) {
+ logCheckingThread.setDaemon(true)
+ logCheckingThread.start()
+ }
}
override def getListing() = applications.values
@@ -115,8 +127,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
override def getAppUI(appId: String): Option[SparkUI] = {
try {
applications.get(appId).map { info =>
- val (replayBus, appListener) = createReplayBus(fs.getFileStatus(
- new Path(logDir, info.logDir)))
+ val replayBus = new ReplayListenerBus()
val ui = {
val conf = this.conf.clone()
val appSecManager = new SecurityManager(conf)
@@ -125,15 +136,17 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
// Do not call ui.bind() to avoid creating a new server for each application
}
- replayBus.replay()
+ val appListener = new ApplicationEventListener()
+ replayBus.addListener(appListener)
+ val appInfo = replay(fs.getFileStatus(new Path(logDir, info.logPath)), replayBus)
- ui.setAppName(s"${appListener.appName.getOrElse(NOT_STARTED)} ($appId)")
+ ui.setAppName(s"${appInfo.name} ($appId)")
val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false)
ui.getSecurityManager.setAcls(uiAclsEnabled)
// make sure to set admin acls before view acls so they are properly picked up
ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse(""))
- ui.getSecurityManager.setViewAcls(appListener.sparkUser.getOrElse(NOT_STARTED),
+ ui.getSecurityManager.setViewAcls(appInfo.sparkUser,
appListener.viewAcls.getOrElse(""))
ui
}
@@ -149,45 +162,38 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
* Tries to reuse as much of the data already in memory as possible, by not reading
* applications that haven't been updated since last time the logs were checked.
*/
- private def checkForLogs() = {
+ private[history] def checkForLogs(): Unit = {
lastLogCheckTimeMs = getMonotonicTimeMs()
logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
- try {
- val logStatus = fs.listStatus(new Path(logDir))
- val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
- // Load all new logs from the log directory. Only directories that have a modification time
- // later than the last known log directory will be loaded.
+ try {
var newLastModifiedTime = lastModifiedTime
- val logInfos = logDirs
- .filter { dir =>
- if (fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))) {
- val modTime = getModificationTime(dir)
+ val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq)
+ .getOrElse(Seq[FileStatus]())
+ val logInfos = statusList
+ .filter { entry =>
+ try {
+ val modTime = getModificationTime(entry)
newLastModifiedTime = math.max(newLastModifiedTime, modTime)
- modTime > lastModifiedTime
- } else {
- false
+ modTime >= lastModifiedTime
+ } catch {
+ case e: AccessControlException =>
+ // Do not use "logInfo" since these messages can get pretty noisy if printed on
+ // every poll.
+ logDebug(s"No permission to read $entry, ignoring.")
+ false
}
}
- .flatMap { dir =>
+ .flatMap { entry =>
try {
- val (replayBus, appListener) = createReplayBus(dir)
- replayBus.replay()
- Some(new FsApplicationHistoryInfo(
- dir.getPath().getName(),
- appListener.appId.getOrElse(dir.getPath().getName()),
- appListener.appName.getOrElse(NOT_STARTED),
- appListener.startTime.getOrElse(-1L),
- appListener.endTime.getOrElse(-1L),
- getModificationTime(dir),
- appListener.sparkUser.getOrElse(NOT_STARTED)))
+ Some(replay(entry, new ReplayListenerBus()))
} catch {
case e: Exception =>
- logInfo(s"Failed to load application log data from $dir.", e)
+ logError(s"Failed to load application log data from $entry.", e)
None
}
}
- .sortBy { info => -info.endTime }
+ .sortBy { info => (-info.endTime, -info.startTime) }
lastModifiedTime = newLastModifiedTime
@@ -217,37 +223,112 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
applications = newApps
}
} catch {
- case t: Throwable => logError("Exception in checking for event log updates", t)
+ case e: Exception => logError("Exception in checking for event log updates", e)
}
}
- private def createReplayBus(logDir: FileStatus): (ReplayListenerBus, ApplicationEventListener) = {
- val path = logDir.getPath()
- val elogInfo = EventLoggingListener.parseLoggingInfo(path, fs)
- val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
- val appListener = new ApplicationEventListener
- replayBus.addListener(appListener)
- (replayBus, appListener)
+ /**
+ * Replays the events in the specified log file and returns information about the associated
+ * application.
+ */
+ private def replay(eventLog: FileStatus, bus: ReplayListenerBus): FsApplicationHistoryInfo = {
+ val logPath = eventLog.getPath()
+ val (logInput, sparkVersion) =
+ if (isLegacyLogDirectory(eventLog)) {
+ openLegacyEventLog(logPath)
+ } else {
+ EventLoggingListener.openEventLog(logPath, fs)
+ }
+ try {
+ val appListener = new ApplicationEventListener
+ bus.addListener(appListener)
+ bus.replay(logInput, sparkVersion)
+ new FsApplicationHistoryInfo(
+ logPath.getName(),
+ appListener.appId.getOrElse(logPath.getName()),
+ appListener.appName.getOrElse(NOT_STARTED),
+ appListener.startTime.getOrElse(-1L),
+ appListener.endTime.getOrElse(-1L),
+ getModificationTime(eventLog),
+ appListener.sparkUser.getOrElse(NOT_STARTED),
+ isApplicationCompleted(eventLog))
+ } finally {
+ logInput.close()
+ }
}
- /** Return when this directory was last modified. */
- private def getModificationTime(dir: FileStatus): Long = {
- try {
- val logFiles = fs.listStatus(dir.getPath)
- if (logFiles != null && !logFiles.isEmpty) {
- logFiles.map(_.getModificationTime).max
- } else {
- dir.getModificationTime
+ /**
+ * Loads a legacy log directory. This assumes that the log directory contains a single event
+ * log file (along with other metadata files), which is the case for directories generated by
+ * the code in previous releases.
+ *
+ * @return 2-tuple of (input stream of the events, version of Spark which wrote the log)
+ */
+ private[history] def openLegacyEventLog(dir: Path): (InputStream, String) = {
+ val children = fs.listStatus(dir)
+ var eventLogPath: Path = null
+ var codecName: Option[String] = None
+ var sparkVersion: String = null
+
+ children.foreach { child =>
+ child.getPath().getName() match {
+ case name if name.startsWith(LOG_PREFIX) =>
+ eventLogPath = child.getPath()
+
+ case codec if codec.startsWith(COMPRESSION_CODEC_PREFIX) =>
+ codecName = Some(codec.substring(COMPRESSION_CODEC_PREFIX.length()))
+
+ case version if version.startsWith(SPARK_VERSION_PREFIX) =>
+ sparkVersion = version.substring(SPARK_VERSION_PREFIX.length())
+
+ case _ =>
}
- } catch {
- case t: Throwable =>
- logError("Exception in accessing modification time of %s".format(dir.getPath), t)
- -1L
+ }
+
+ if (eventLogPath == null || sparkVersion == null) {
+ throw new IllegalArgumentException(s"$dir is not a Spark application log directory.")
+ }
+
+ val codec = try {
+ codecName.map { c => CompressionCodec.createCodec(conf, c) }
+ } catch {
+ case e: Exception =>
+ throw new IllegalArgumentException(s"Unknown compression codec $codecName.")
+ }
+
+ val in = new BufferedInputStream(fs.open(eventLogPath))
+ (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion)
+ }
+
+ /**
+ * Return whether the specified event log path contains a old directory-based event log.
+ * Previously, the event log of an application comprises of multiple files in a directory.
+ * As of Spark 1.3, these files are consolidated into a single one that replaces the directory.
+ * See SPARK-2261 for more detail.
+ */
+ private def isLegacyLogDirectory(entry: FileStatus): Boolean = entry.isDir()
+
+ private def getModificationTime(fsEntry: FileStatus): Long = {
+ if (fsEntry.isDir) {
+ fs.listStatus(fsEntry.getPath).map(_.getModificationTime()).max
+ } else {
+ fsEntry.getModificationTime()
}
}
/** Returns the system's mononotically increasing time. */
- private def getMonotonicTimeMs() = System.nanoTime() / (1000 * 1000)
+ private def getMonotonicTimeMs(): Long = System.nanoTime() / (1000 * 1000)
+
+ /**
+ * Return true when the application has completed.
+ */
+ private def isApplicationCompleted(entry: FileStatus): Boolean = {
+ if (isLegacyLogDirectory(entry)) {
+ fs.exists(new Path(entry.getPath(), APPLICATION_COMPLETE))
+ } else {
+ !entry.getPath().getName().endsWith(EventLoggingListener.IN_PROGRESS)
+ }
+ }
}
@@ -256,11 +337,12 @@ private object FsHistoryProvider {
}
private class FsApplicationHistoryInfo(
- val logDir: String,
+ val logPath: String,
id: String,
name: String,
startTime: Long,
endTime: Long,
lastUpdated: Long,
- sparkUser: String)
- extends ApplicationHistoryInfo(id, name, startTime, endTime, lastUpdated, sparkUser)
+ sparkUser: String,
+ completed: Boolean = true)
+ extends ApplicationHistoryInfo(id, name, startTime, endTime, lastUpdated, sparkUser, completed)
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
index 0d5dcfb1ddffe..e4e7bc2216014 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
@@ -31,8 +31,10 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
def render(request: HttpServletRequest): Seq[Node] = {
val requestedPage = Option(request.getParameter("page")).getOrElse("1").toInt
val requestedFirst = (requestedPage - 1) * pageSize
+ val requestedIncomplete =
+ Option(request.getParameter("showIncomplete")).getOrElse("false").toBoolean
- val allApps = parent.getApplicationList()
+ val allApps = parent.getApplicationList().filter(_.completed != requestedIncomplete)
val actualFirst = if (requestedFirst < allApps.size) requestedFirst else 0
val apps = allApps.slice(actualFirst, Math.min(actualFirst + pageSize, allApps.size))
@@ -65,25 +67,26 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
Showing {actualFirst + 1}-{last + 1} of {allApps.size}
-
- {
- if (actualPage > 1) {
- <
- 1
- }
+ {if (requestedIncomplete) "(Incomplete applications)"}
+
+ {
+ if (actualPage > 1) {
+ <
+ 1
}
- {if (actualPage - plusOrMinus > secondPageFromLeft) " ... "}
- {leftSideIndices}
- {actualPage}
- {rightSideIndices}
- {if (actualPage + plusOrMinus < secondPageFromRight) " ... "}
- {
- if (actualPage < pageCount) {
- {pageCount}
- >
- }
+ }
+ {if (actualPage - plusOrMinus > secondPageFromLeft) " ... "}
+ {leftSideIndices}
+ {actualPage}
+ {rightSideIndices}
+ {if (actualPage + plusOrMinus < secondPageFromRight) " ... "}
+ {
+ if (actualPage < pageCount) {
+ {pageCount}
+ >
}
-
+ }
+
++
appTable
} else {
@@ -96,6 +99,15 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
}
}
+
+ {
+ if (requestedIncomplete) {
+ "Back to completed applications"
+ } else {
+ "Show incomplete applications"
+ }
+ }
+
UIUtils.basicSparkPage(content, "History Server")
@@ -117,8 +129,9 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
private def appRow(info: ApplicationHistoryInfo): Seq[Node] = {
val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}"
val startTime = UIUtils.formatDate(info.startTime)
- val endTime = UIUtils.formatDate(info.endTime)
- val duration = UIUtils.formatDuration(info.endTime - info.startTime)
+ val endTime = if (info.endTime > 0) UIUtils.formatDate(info.endTime) else "-"
+ val duration =
+ if (info.endTime > 0) UIUtils.formatDuration(info.endTime - info.startTime) else "-"
val lastUpdated = UIUtils.formatDate(info.lastUpdated)
{info.id} |
@@ -130,4 +143,11 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
{lastUpdated} |
}
+
+ private def makePageLink(linkPage: Int, showIncomplete: Boolean): String = {
+ "/?" + Array(
+ "page=" + linkPage,
+ "showIncomplete=" + showIncomplete
+ ).mkString("&")
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index ce00c0ffd21e0..fa9bfe5426b6c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -158,11 +158,12 @@ class HistoryServer(
/**
* The recommended way of starting and stopping a HistoryServer is through the scripts
- * start-history-server.sh and stop-history-server.sh. The path to a base log directory
- * is must be specified, while the requested UI port is optional. For example:
+ * start-history-server.sh and stop-history-server.sh. The path to a base log directory,
+ * as well as any other relevant history server configuration, should be specified via
+ * the $SPARK_HISTORY_OPTS environment variable. For example:
*
- * ./sbin/spark-history-server.sh /tmp/spark-events
- * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events
+ * export SPARK_HISTORY_OPTS="-Dspark.history.fs.logDirectory=/tmp/spark-events"
+ * ./sbin/start-history-server.sh
*
* This launches the HistoryServer as a Spark daemon.
*/
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 7b32c505def9b..4b631ec639071 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -17,6 +17,7 @@
package org.apache.spark.deploy.master
+import java.io.FileNotFoundException
import java.net.URLEncoder
import java.text.SimpleDateFormat
import java.util.Date
@@ -32,6 +33,7 @@ import akka.pattern.ask
import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
import akka.serialization.Serialization
import akka.serialization.SerializationExtension
+import org.apache.hadoop.fs.Path
import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException}
import org.apache.spark.deploy.{ApplicationDescription, DriverDescription,
@@ -56,6 +58,7 @@ private[spark] class Master(
import context.dispatcher // to use Akka's scheduler.schedule()
val conf = new SparkConf
+ val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf)
def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs
val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000
@@ -120,6 +123,7 @@ private[spark] class Master(
override def preStart() {
logInfo("Starting Spark master at " + masterUrl)
+ logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}")
// Listen for remote client disconnection events, since they don't go through Akka's watch()
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
webUi.bind()
@@ -129,6 +133,10 @@ private[spark] class Master(
masterMetricsSystem.registerSource(masterSource)
masterMetricsSystem.start()
applicationMetricsSystem.start()
+ // Attach the master and app metrics servlet handler to the web ui after the metrics systems are
+ // started.
+ masterMetricsSystem.getServletHandlers.foreach(webUi.attachHandler)
+ applicationMetricsSystem.getServletHandlers.foreach(webUi.attachHandler)
val (persistenceEngine_, leaderElectionAgent_) = RECOVERY_MODE match {
case "ZOOKEEPER" =>
@@ -510,7 +518,7 @@ private[spark] class Master(
val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE))
val numWorkersAlive = shuffledAliveWorkers.size
var curPos = 0
-
+
for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers
// We assign workers to each waiting driver in a round-robin fashion. For each driver, we
// start from the last worker that was assigned a driver, and continue onwards until we have
@@ -697,6 +705,11 @@ private[spark] class Master(
}
persistenceEngine.removeApplication(app)
schedule()
+
+ // Tell all workers that the application has finished, so they can clean up any app state.
+ workers.foreach { w =>
+ w.actor ! ApplicationFinished(app.id)
+ }
}
}
@@ -707,41 +720,51 @@ private[spark] class Master(
def rebuildSparkUI(app: ApplicationInfo): Boolean = {
val appName = app.desc.name
val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found"
- val eventLogDir = app.desc.eventLogDir.getOrElse {
- // Event logging is not enabled for this application
- app.desc.appUiUrl = notFoundBasePath
- return false
- }
-
- val appEventLogDir = EventLoggingListener.getLogDirPath(eventLogDir, app.id)
- val fileSystem = Utils.getHadoopFileSystem(appEventLogDir,
- SparkHadoopUtil.get.newConfiguration(conf))
- val eventLogInfo = EventLoggingListener.parseLoggingInfo(appEventLogDir, fileSystem)
- val eventLogPaths = eventLogInfo.logPaths
- val compressionCodec = eventLogInfo.compressionCodec
-
- if (eventLogPaths.isEmpty) {
- // Event logging is enabled for this application, but no event logs are found
- val title = s"Application history not found (${app.id})"
- var msg = s"No event logs found for application $appName in $appEventLogDir."
- logWarning(msg)
- msg += " Did you specify the correct logging directory?"
- msg = URLEncoder.encode(msg, "UTF-8")
- app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title"
- return false
- }
-
try {
- val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec)
+ val eventLogFile = app.desc.eventLogDir
+ .map { dir => EventLoggingListener.getLogPath(dir, app.id) }
+ .getOrElse {
+ // Event logging is not enabled for this application
+ app.desc.appUiUrl = notFoundBasePath
+ return false
+ }
+
+ val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf)
+
+ if (fs.exists(new Path(eventLogFile + EventLoggingListener.IN_PROGRESS))) {
+ // Event logging is enabled for this application, but the application is still in progress
+ val title = s"Application history not found (${app.id})"
+ var msg = s"Application $appName is still in progress."
+ logWarning(msg)
+ msg = URLEncoder.encode(msg, "UTF-8")
+ app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title"
+ return false
+ }
+
+ val (logInput, sparkVersion) = EventLoggingListener.openEventLog(new Path(eventLogFile), fs)
+ val replayBus = new ReplayListenerBus()
val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf),
appName + " (completed)", HistoryServer.UI_PATH_PREFIX + s"/${app.id}")
- replayBus.replay()
+ try {
+ replayBus.replay(logInput, sparkVersion)
+ } finally {
+ logInput.close()
+ }
appIdToUI(app.id) = ui
webUi.attachSparkUI(ui)
// Application UI is successfully rebuilt, so link the Master UI to it
- app.desc.appUiUrl = ui.getBasePath
+ app.desc.appUiUrl = ui.basePath
true
} catch {
+ case fnf: FileNotFoundException =>
+ // Event logging is enabled for this application, but no event logs are found
+ val title = s"Application history not found (${app.id})"
+ var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir}."
+ logWarning(msg)
+ msg += " Did you specify the correct logging directory?"
+ msg = URLEncoder.encode(msg, "UTF-8")
+ app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title"
+ false
case e: Exception =>
// Relay exception message to application UI page
val title = s"Application history load error (${app.id})"
@@ -823,7 +846,6 @@ private[spark] class Master(
private[spark] object Master extends Logging {
val systemName = "sparkMaster"
private val actorName = "Master"
- val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
def main(argStrings: Array[String]) {
SignalLogger.register(log)
@@ -833,14 +855,24 @@ private[spark] object Master extends Logging {
actorSystem.awaitTermination()
}
- /** Returns an `akka.tcp://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */
+ /**
+ * Returns an `akka.tcp://...` URL for the Master actor given a sparkUrl `spark://host:port`.
+ *
+ * @throws SparkException if the url is invalid
+ */
def toAkkaUrl(sparkUrl: String): String = {
- sparkUrl match {
- case sparkUrlRegex(host, port) =>
- "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName)
- case _ =>
- throw new SparkException("Invalid master URL: " + sparkUrl)
- }
+ val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl)
+ "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName)
+ }
+
+ /**
+ * Returns an akka `Address` for the Master actor given a sparkUrl `spark://host:port`.
+ *
+ * @throws SparkException if the url is invalid
+ */
+ def toAkkaAddress(sparkUrl: String): Address = {
+ val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl)
+ Address("akka.tcp", systemName, host, port)
}
def startSystemAndActor(
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index d86ec1e03e45c..73400c5affb5d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -41,8 +41,6 @@ class MasterWebUI(val master: Master, requestedPort: Int)
attachPage(new HistoryNotFoundPage(this))
attachPage(new MasterPage(this))
attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"))
- master.masterMetricsSystem.getServletHandlers.foreach(attachHandler)
- master.applicationMetricsSystem.getServletHandlers.foreach(attachHandler)
}
/** Attach a reconstructed UI to this Master UI. Only valid after bind(). */
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index f4fedc6327ab9..acbdf0d8bd7bc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -47,6 +47,7 @@ private[spark] class ExecutorRunner(
val executorDir: File,
val workerUrl: String,
val conf: SparkConf,
+ val appLocalDirs: Seq[String],
var state: ExecutorState.Value)
extends Logging {
@@ -77,7 +78,7 @@ private[spark] class ExecutorRunner(
/**
* Kill executor process, wait for exit and notify worker to update resource status.
*
- * @param message the exception message which caused the executor's death
+ * @param message the exception message which caused the executor's death
*/
private def killProcess(message: Option[String]) {
var exitCode: Option[Int] = None
@@ -129,6 +130,7 @@ private[spark] class ExecutorRunner(
logInfo("Launch command: " + command.mkString("\"", "\" \"", "\""))
builder.directory(executorDir)
+ builder.environment.put("SPARK_LOCAL_DIRS", appLocalDirs.mkString(","))
// In case we are running this from within the Spark Shell, avoid creating a "scala"
// parent process for the executor command
builder.environment.put("SPARK_LAUNCH_WITH_SCALA", "0")
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index eb11163538b20..13599830123d0 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -23,7 +23,7 @@ import java.text.SimpleDateFormat
import java.util.{UUID, Date}
import scala.collection.JavaConversions._
-import scala.collection.mutable.HashMap
+import scala.collection.mutable.{HashMap, HashSet}
import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.Random
@@ -40,7 +40,7 @@ import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils}
/**
- * @param masterUrls Each url should look like spark://host:port.
+ * @param masterAkkaUrls Each url should be a valid akka url.
*/
private[spark] class Worker(
host: String,
@@ -48,7 +48,7 @@ private[spark] class Worker(
webUiPort: Int,
cores: Int,
memory: Int,
- masterUrls: Array[String],
+ masterAkkaUrls: Array[String],
actorSystemName: String,
actorName: String,
workDirPath: String = null,
@@ -109,6 +109,8 @@ private[spark] class Worker(
val finishedExecutors = new HashMap[String, ExecutorRunner]
val drivers = new HashMap[String, DriverRunner]
val finishedDrivers = new HashMap[String, DriverRunner]
+ val appDirectories = new HashMap[String, Seq[String]]
+ val finishedApps = new HashSet[String]
// The shuffle service is not actually started unless configured.
val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr)
@@ -153,6 +155,7 @@ private[spark] class Worker(
assert(!registered)
logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format(
host, port, cores, Utils.megabytesToString(memory)))
+ logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}")
logInfo("Spark home: " + sparkHome)
createWorkDir()
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
@@ -163,18 +166,16 @@ private[spark] class Worker(
metricsSystem.registerSource(workerSource)
metricsSystem.start()
+ // Attach the worker metrics servlet handler to the web ui after the metrics system is started.
+ metricsSystem.getServletHandlers.foreach(webUi.attachHandler)
}
def changeMaster(url: String, uiUrl: String) {
+ // activeMasterUrl it's a valid Spark url since we receive it from master.
activeMasterUrl = url
activeMasterWebUiUrl = uiUrl
master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
- masterAddress = activeMasterUrl match {
- case Master.sparkUrlRegex(_host, _port) =>
- Address("akka.tcp", Master.systemName, _host, _port.toInt)
- case x =>
- throw new SparkException("Invalid spark URL: " + x)
- }
+ masterAddress = Master.toAkkaAddress(activeMasterUrl)
connected = true
// Cancel any outstanding re-registration attempts because we found a new master
registrationRetryTimer.foreach(_.cancel())
@@ -182,9 +183,9 @@ private[spark] class Worker(
}
private def tryRegisterAllMasters() {
- for (masterUrl <- masterUrls) {
- logInfo("Connecting to master " + masterUrl + "...")
- val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
+ for (masterAkkaUrl <- masterAkkaUrls) {
+ logInfo("Connecting to master " + masterAkkaUrl + "...")
+ val actor = context.actorSelection(masterAkkaUrl)
actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort, publicAddress)
}
}
@@ -292,7 +293,7 @@ private[spark] class Worker(
val isAppStillRunning = executors.values.map(_.appId).contains(appIdFromDir)
dir.isDirectory && !isAppStillRunning &&
!Utils.doesDirectoryContainAnyNewFiles(dir, APP_DATA_RETENTION_SECS)
- }.foreach { dir =>
+ }.foreach { dir =>
logInfo(s"Removing directory: ${dir.getPath}")
Utils.deleteRecursively(dir)
}
@@ -337,8 +338,19 @@ private[spark] class Worker(
throw new IOException("Failed to create directory " + executorDir)
}
+ // Create local dirs for the executor. These are passed to the executor via the
+ // SPARK_LOCAL_DIRS environment variable, and deleted by the Worker when the
+ // application finishes.
+ val appLocalDirs = appDirectories.get(appId).getOrElse {
+ Utils.getOrCreateLocalRootDirs(conf).map { dir =>
+ Utils.createDirectory(dir).getAbsolutePath()
+ }.toSeq
+ }
+ appDirectories(appId) = appLocalDirs
+
val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_,
- self, workerId, host, sparkHome, executorDir, akkaUrl, conf, ExecutorState.LOADING)
+ self, workerId, host, sparkHome, executorDir, akkaUrl, conf, appLocalDirs,
+ ExecutorState.LOADING)
executors(appId + "/" + execId) = manager
manager.start()
coresUsed += cores_
@@ -375,6 +387,7 @@ private[spark] class Worker(
message.map(" message " + _).getOrElse("") +
exitStatus.map(" exitStatus " + _).getOrElse(""))
}
+ maybeCleanupApplication(appId)
}
case KillExecutor(masterUrl, appId, execId) =>
@@ -444,6 +457,9 @@ private[spark] class Worker(
case ReregisterWithMaster =>
reregisterWithMaster()
+ case ApplicationFinished(id) =>
+ finishedApps += id
+ maybeCleanupApplication(id)
}
private def masterDisconnected() {
@@ -452,6 +468,19 @@ private[spark] class Worker(
registerWithMaster()
}
+ private def maybeCleanupApplication(id: String): Unit = {
+ val shouldCleanup = finishedApps.contains(id) && !executors.values.exists(_.appId == id)
+ if (shouldCleanup) {
+ finishedApps -= id
+ appDirectories.remove(id).foreach { dirList =>
+ logInfo(s"Cleaning up local directories for application $id")
+ dirList.foreach { dir =>
+ Utils.deleteRecursively(new File(dir))
+ }
+ }
+ }
+ }
+
def generateWorkerId(): String = {
"worker-%s-%s-%d".format(createDateFormat.format(new Date), host, port)
}
@@ -494,8 +523,9 @@ private[spark] object Worker extends Logging {
val securityMgr = new SecurityManager(conf)
val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port,
conf = conf, securityManager = securityMgr)
+ val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl)
actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
- masterUrls, systemName, actorName, workDir, conf, securityMgr), name = actorName)
+ masterAkkaUrls, systemName, actorName, workDir, conf, securityMgr), name = actorName)
(actorSystem, boundPort)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index b07942a9ca729..7ac81a2d87efd 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -50,7 +50,6 @@ class WorkerWebUI(
attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static"))
attachHandler(createServletHandler("/log",
(request: HttpServletRequest) => logPage.renderLog(request), worker.securityMgr))
- worker.metricsSystem.getServletHandlers.foreach(attachHandler)
}
}
diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index 5f46f3b1f085e..c794a7bc3599e 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer
import scala.concurrent.Await
-import akka.actor.{Actor, ActorSelection, ActorSystem, Props}
+import akka.actor.{Actor, ActorSelection, Props}
import akka.pattern.Patterns
import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent}
@@ -38,8 +38,7 @@ private[spark] class CoarseGrainedExecutorBackend(
executorId: String,
hostPort: String,
cores: Int,
- sparkProperties: Seq[(String, String)],
- actorSystem: ActorSystem)
+ env: SparkEnv)
extends Actor with ActorLogReceive with ExecutorBackend with Logging {
Utils.checkHostPort(hostPort, "Expected hostport")
@@ -58,8 +57,7 @@ private[spark] class CoarseGrainedExecutorBackend(
case RegisteredExecutor =>
logInfo("Successfully registered with driver")
val (hostname, _) = Utils.parseHostPort(hostPort)
- executor = new Executor(executorId, hostname, sparkProperties, cores, isLocal = false,
- actorSystem)
+ executor = new Executor(executorId, hostname, env, isLocal = false)
case RegisterExecutorFailed(message) =>
logError("Slave registration failed: " + message)
@@ -70,7 +68,7 @@ private[spark] class CoarseGrainedExecutorBackend(
logError("Received LaunchTask command but executor was null")
System.exit(1)
} else {
- val ser = SparkEnv.get.closureSerializer.newInstance()
+ val ser = env.closureSerializer.newInstance()
val taskDesc = ser.deserialize[TaskDescription](data.value)
logInfo("Got assigned task " + taskDesc.taskId)
executor.launchTask(this, taskDesc.taskId, taskDesc.name, taskDesc.serializedTask)
@@ -128,21 +126,25 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
Seq[(String, String)](("spark.app.id", appId))
fetcher.shutdown()
- // Create a new ActorSystem using driver's Spark properties to run the backend.
+ // Create SparkEnv using properties we fetched from the driver.
val driverConf = new SparkConf().setAll(props)
- val (actorSystem, boundPort) = AkkaUtils.createActorSystem(
- SparkEnv.executorActorSystemName,
- hostname, port, driverConf, new SecurityManager(driverConf))
- // set it
+ val env = SparkEnv.createExecutorEnv(
+ driverConf, executorId, hostname, port, cores, isLocal = false)
+
+ // SparkEnv sets spark.driver.port so it shouldn't be 0 anymore.
+ val boundPort = env.conf.getInt("spark.executor.port", 0)
+ assert(boundPort != 0)
+
+ // Start the CoarseGrainedExecutorBackend actor.
val sparkHostPort = hostname + ":" + boundPort
- actorSystem.actorOf(
+ env.actorSystem.actorOf(
Props(classOf[CoarseGrainedExecutorBackend],
- driverUrl, executorId, sparkHostPort, cores, props, actorSystem),
+ driverUrl, executorId, sparkHostPort, cores, env),
name = "Executor")
workerUrl.foreach { url =>
- actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher")
+ env.actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher")
}
- actorSystem.awaitTermination()
+ env.actorSystem.awaitTermination()
}
}
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index d21c63fa53f81..3a5d2638a32e3 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -26,7 +26,7 @@ import scala.collection.JavaConversions._
import scala.collection.mutable.{ArrayBuffer, HashMap}
import scala.util.control.NonFatal
-import akka.actor.{Props, ActorSystem}
+import akka.actor.Props
import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
@@ -42,10 +42,8 @@ import org.apache.spark.util.{SparkUncaughtExceptionHandler, AkkaUtils, Utils}
private[spark] class Executor(
executorId: String,
slaveHostname: String,
- properties: Seq[(String, String)],
- numCores: Int,
- isLocal: Boolean = false,
- actorSystem: ActorSystem = null)
+ env: SparkEnv,
+ isLocal: Boolean = false)
extends Logging
{
// Application dependencies (added through SparkContext) that we've fetched so far on this node.
@@ -55,6 +53,8 @@ private[spark] class Executor(
private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0))
+ private val conf = env.conf
+
@volatile private var isStopped = false
// No ip or host:port - just hostname
@@ -65,10 +65,6 @@ private[spark] class Executor(
// Make sure the local hostname we report matches the cluster scheduler's name for this host
Utils.setCustomHostname(slaveHostname)
- // Set spark.* properties from executor arg
- val conf = new SparkConf(true)
- conf.setAll(properties)
-
if (!isLocal) {
// Setup an uncaught exception handler for non-local mode.
// Make any thread terminations due to uncaught exceptions kill the entire
@@ -77,21 +73,11 @@ private[spark] class Executor(
}
val executorSource = new ExecutorSource(this, executorId)
-
- // Initialize Spark environment (using system properties read above)
conf.set("spark.executor.id", executorId)
- private val env = {
- if (!isLocal) {
- val port = conf.getInt("spark.executor.port", 0)
- val _env = SparkEnv.createExecutorEnv(
- conf, executorId, slaveHostname, port, numCores, isLocal, actorSystem)
- SparkEnv.set(_env)
- _env.metricsSystem.registerSource(executorSource)
- _env.blockManager.initialize(conf.getAppId)
- _env
- } else {
- SparkEnv.get
- }
+
+ if (!isLocal) {
+ env.metricsSystem.registerSource(executorSource)
+ env.blockManager.initialize(conf.getAppId)
}
// Create an actor for receiving RPCs from the driver
@@ -145,6 +131,8 @@ private[spark] class Executor(
}
}
+ private def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum
+
class TaskRunner(
execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer)
extends Runnable {
@@ -152,6 +140,7 @@ private[spark] class Executor(
@volatile private var killed = false
@volatile var task: Task[Any] = _
@volatile var attemptedTask: Option[Task[Any]] = None
+ @volatile var startGCTime: Long = _
def kill(interruptThread: Boolean) {
logInfo(s"Executor is trying to kill $taskName (TID $taskId)")
@@ -164,12 +153,11 @@ private[spark] class Executor(
override def run() {
val deserializeStartTime = System.currentTimeMillis()
Thread.currentThread.setContextClassLoader(replClassLoader)
- val ser = SparkEnv.get.closureSerializer.newInstance()
+ val ser = env.closureSerializer.newInstance()
logInfo(s"Running $taskName (TID $taskId)")
execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER)
var taskStart: Long = 0
- def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum
- val startGCTime = gcTime
+ startGCTime = gcTime
try {
val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask)
@@ -200,7 +188,7 @@ private[spark] class Executor(
throw new TaskKilledException
}
- val resultSer = SparkEnv.get.serializer.newInstance()
+ val resultSer = env.serializer.newInstance()
val beforeSerialization = System.currentTimeMillis()
val valueBytes = resultSer.serialize(value)
val afterSerialization = System.currentTimeMillis()
@@ -376,10 +364,13 @@ private[spark] class Executor(
while (!isStopped) {
val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]()
+ val curGCTime = gcTime
+
for (taskRunner <- runningTasks.values()) {
if (!taskRunner.attemptedTask.isEmpty) {
Option(taskRunner.task).flatMap(_.metrics).foreach { metrics =>
metrics.updateShuffleReadMetrics
+ metrics.jvmGCTime = curGCTime - taskRunner.startGCTime
if (isLocal) {
// JobProgressListener will hold an reference of it during
// onExecutorMetricsUpdate(), then JobProgressListener can not see
diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
index f15e6bc33fb41..2e23ae0a4f831 100644
--- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
@@ -22,10 +22,10 @@ import java.nio.ByteBuffer
import scala.collection.JavaConversions._
import org.apache.mesos.protobuf.ByteString
-import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary}
+import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver}
import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _}
-import org.apache.spark.{Logging, TaskState}
+import org.apache.spark.{Logging, TaskState, SparkConf, SparkEnv}
import org.apache.spark.TaskState.TaskState
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.util.{SignalLogger, Utils}
@@ -64,11 +64,15 @@ private[spark] class MesosExecutorBackend
this.driver = driver
val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++
Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue))
+ val conf = new SparkConf(loadDefaults = true).setAll(properties)
+ val port = conf.getInt("spark.executor.port", 0)
+ val env = SparkEnv.createExecutorEnv(
+ conf, executorId, slaveInfo.getHostname, port, cpusPerTask, isLocal = false)
+
executor = new Executor(
executorId,
slaveInfo.getHostname,
- properties,
- cpusPerTask)
+ env)
}
override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) {
@@ -76,7 +80,9 @@ private[spark] class MesosExecutorBackend
if (executor == null) {
logError("Received launchTask but executor was null")
} else {
- executor.launchTask(this, taskId, taskInfo.getName, taskInfo.getData.asReadOnlyByteBuffer)
+ SparkHadoopUtil.get.runAsSparkUser { () =>
+ executor.launchTask(this, taskId, taskInfo.getName, taskInfo.getData.asReadOnlyByteBuffer)
+ }
}
}
@@ -108,11 +114,8 @@ private[spark] class MesosExecutorBackend
private[spark] object MesosExecutorBackend extends Logging {
def main(args: Array[String]) {
SignalLogger.register(log)
- SparkHadoopUtil.get.runAsSparkUser { () =>
- MesosNativeLibrary.load()
- // Create a new Executor and start it running
- val runner = new MesosExecutorBackend()
- new MesosExecutorDriver(runner).run()
- }
+ // Create a new Executor and start it running
+ val runner = new MesosExecutorBackend()
+ new MesosExecutorDriver(runner).run()
}
}
diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala
index 89b29af2000c8..c219d21fbefa9 100644
--- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala
+++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala
@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.{BytesWritable, LongWritable}
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext}
+import org.apache.spark.deploy.SparkHadoopUtil
/**
* Custom Input Format for reading and splitting flat binary files that contain records,
@@ -33,7 +34,7 @@ private[spark] object FixedLengthBinaryInputFormat {
/** Retrieves the record length property from a Hadoop configuration */
def getRecordLength(context: JobContext): Int = {
- context.getConfiguration.get(RECORD_LENGTH_PROPERTY).toInt
+ SparkHadoopUtil.get.getConfigurationFromJobContext(context).get(RECORD_LENGTH_PROPERTY).toInt
}
}
diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
index 36a1e5d475f46..67a96925da019 100644
--- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
+++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala
@@ -24,6 +24,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory
import org.apache.hadoop.io.{BytesWritable, LongWritable}
import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext}
import org.apache.hadoop.mapreduce.lib.input.FileSplit
+import org.apache.spark.deploy.SparkHadoopUtil
/**
* FixedLengthBinaryRecordReader is returned by FixedLengthBinaryInputFormat.
@@ -82,7 +83,7 @@ private[spark] class FixedLengthBinaryRecordReader
// the actual file we will be reading from
val file = fileSplit.getPath
// job configuration
- val job = context.getConfiguration
+ val job = SparkHadoopUtil.get.getConfigurationFromJobContext(context)
// check compression
val codec = new CompressionCodecFactory(job).getCodec(file)
if (codec != null) {
diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
index 457472547fcbb..593a62b3e3b32 100644
--- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
+++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
@@ -28,6 +28,7 @@ import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAt
import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat, CombineFileRecordReader, CombineFileSplit}
import org.apache.spark.annotation.Experimental
+import org.apache.spark.deploy.SparkHadoopUtil
/**
* A general format for reading whole files in as streams, byte arrays,
@@ -145,7 +146,8 @@ class PortableDataStream(
private val confBytes = {
val baos = new ByteArrayOutputStream()
- context.getConfiguration.write(new DataOutputStream(baos))
+ SparkHadoopUtil.get.getConfigurationFromJobContext(context).
+ write(new DataOutputStream(baos))
baos.toByteArray
}
diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala
index d3601cca832b2..aaef7c74eea33 100644
--- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala
+++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala
@@ -19,7 +19,6 @@ package org.apache.spark.input
import scala.collection.JavaConversions._
-import org.apache.hadoop.conf.{Configuration, Configurable}
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.InputSplit
import org.apache.hadoop.mapreduce.JobContext
@@ -38,18 +37,13 @@ private[spark] class WholeTextFileInputFormat
override protected def isSplitable(context: JobContext, file: Path): Boolean = false
- private var conf: Configuration = _
- def setConf(c: Configuration) {
- conf = c
- }
- def getConf: Configuration = conf
-
override def createRecordReader(
split: InputSplit,
context: TaskAttemptContext): RecordReader[String, String] = {
- val reader = new WholeCombineFileRecordReader(split, context)
- reader.setConf(conf)
+ val reader =
+ new ConfigurableCombineFileRecordReader(split, context, classOf[WholeTextFileRecordReader])
+ reader.setConf(getConf)
reader
}
diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala
index 6d59b24eb0596..31bde8a78f3c6 100644
--- a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala
+++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala
@@ -17,7 +17,7 @@
package org.apache.spark.input
-import org.apache.hadoop.conf.{Configuration, Configurable}
+import org.apache.hadoop.conf.{Configuration, Configurable => HConfigurable}
import com.google.common.io.{ByteStreams, Closeables}
import org.apache.hadoop.io.Text
@@ -26,6 +26,19 @@ import org.apache.hadoop.mapreduce.InputSplit
import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, CombineFileRecordReader}
import org.apache.hadoop.mapreduce.RecordReader
import org.apache.hadoop.mapreduce.TaskAttemptContext
+import org.apache.spark.deploy.SparkHadoopUtil
+
+
+/**
+ * A trait to implement [[org.apache.hadoop.conf.Configurable Configurable]] interface.
+ */
+private[spark] trait Configurable extends HConfigurable {
+ private var conf: Configuration = _
+ def setConf(c: Configuration) {
+ conf = c
+ }
+ def getConf: Configuration = conf
+}
/**
* A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file
@@ -38,14 +51,9 @@ private[spark] class WholeTextFileRecordReader(
index: Integer)
extends RecordReader[String, String] with Configurable {
- private var conf: Configuration = _
- def setConf(c: Configuration) {
- conf = c
- }
- def getConf: Configuration = conf
-
private[this] val path = split.getPath(index)
- private[this] val fs = path.getFileSystem(context.getConfiguration)
+ private[this] val fs = path.getFileSystem(
+ SparkHadoopUtil.get.getConfigurationFromJobContext(context))
// True means the current file has been processed, then skip it.
private[this] var processed = false
@@ -87,29 +95,24 @@ private[spark] class WholeTextFileRecordReader(
/**
- * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file
- * out in a key-value pair, where the key is the file path and the value is the entire content of
- * the file.
+ * A [[org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader CombineFileRecordReader]]
+ * that can pass Hadoop Configuration to [[org.apache.hadoop.conf.Configurable Configurable]]
+ * RecordReaders.
*/
-private[spark] class WholeCombineFileRecordReader(
+private[spark] class ConfigurableCombineFileRecordReader[K, V](
split: InputSplit,
- context: TaskAttemptContext)
- extends CombineFileRecordReader[String, String](
+ context: TaskAttemptContext,
+ recordReaderClass: Class[_ <: RecordReader[K, V] with HConfigurable])
+ extends CombineFileRecordReader[K, V](
split.asInstanceOf[CombineFileSplit],
context,
- classOf[WholeTextFileRecordReader]
+ recordReaderClass
) with Configurable {
- private var conf: Configuration = _
- def setConf(c: Configuration) {
- conf = c
- }
- def getConf: Configuration = conf
-
override def initNextRecordReader(): Boolean = {
val r = super.initNextRecordReader()
if (r) {
- this.curReader.asInstanceOf[WholeTextFileRecordReader].setConf(conf)
+ this.curReader.asInstanceOf[HConfigurable].setConf(getConf)
}
r
}
diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 1ac7f4e448eb1..f856890d279f4 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -21,11 +21,12 @@ import java.io.{InputStream, OutputStream}
import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream}
-import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream}
+import org.xerial.snappy.{Snappy, SnappyInputStream, SnappyOutputStream}
import org.apache.spark.SparkConf
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.util.Utils
+import org.apache.spark.Logging
/**
* :: DeveloperApi ::
@@ -44,25 +45,33 @@ trait CompressionCodec {
def compressedInputStream(s: InputStream): InputStream
}
-
private[spark] object CompressionCodec {
+ private val configKey = "spark.io.compression.codec"
private val shortCompressionCodecNames = Map(
"lz4" -> classOf[LZ4CompressionCodec].getName,
"lzf" -> classOf[LZFCompressionCodec].getName,
"snappy" -> classOf[SnappyCompressionCodec].getName)
def createCodec(conf: SparkConf): CompressionCodec = {
- createCodec(conf, conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC))
+ createCodec(conf, conf.get(configKey, DEFAULT_COMPRESSION_CODEC))
}
def createCodec(conf: SparkConf, codecName: String): CompressionCodec = {
val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName)
- val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader)
- .getConstructor(classOf[SparkConf])
- ctor.newInstance(conf).asInstanceOf[CompressionCodec]
+ val codec = try {
+ val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader)
+ .getConstructor(classOf[SparkConf])
+ Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec])
+ } catch {
+ case e: ClassNotFoundException => None
+ case e: IllegalArgumentException => None
+ }
+ codec.getOrElse(throw new IllegalArgumentException(s"Codec [$codecName] is not available. " +
+ s"Consider setting $configKey=$FALLBACK_COMPRESSION_CODEC"))
}
+ val FALLBACK_COMPRESSION_CODEC = "lzf"
val DEFAULT_COMPRESSION_CODEC = "snappy"
val ALL_COMPRESSION_CODECS = shortCompressionCodecNames.values.toSeq
}
@@ -120,6 +129,12 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec {
@DeveloperApi
class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec {
+ try {
+ Snappy.getNativeLibraryVersion
+ } catch {
+ case e: Error => throw new IllegalArgumentException
+ }
+
override def compressedOutputStream(s: OutputStream): OutputStream = {
val blockSize = conf.getInt("spark.io.compression.snappy.block.size", 32768)
new SnappyOutputStream(s, blockSize)
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index 5dd67b0cbf683..45633e3de01dd 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -76,22 +76,36 @@ private[spark] class MetricsSystem private (
private val sources = new mutable.ArrayBuffer[Source]
private val registry = new MetricRegistry()
+ private var running: Boolean = false
+
// Treat MetricsServlet as a special sink as it should be exposed to add handlers to web ui
private var metricsServlet: Option[MetricsServlet] = None
- /** Get any UI handlers used by this metrics system. */
- def getServletHandlers = metricsServlet.map(_.getHandlers).getOrElse(Array())
+ /**
+ * Get any UI handlers used by this metrics system; can only be called after start().
+ */
+ def getServletHandlers = {
+ require(running, "Can only call getServletHandlers on a running MetricsSystem")
+ metricsServlet.map(_.getHandlers).getOrElse(Array())
+ }
metricsConfig.initialize()
def start() {
+ require(!running, "Attempting to start a MetricsSystem that is already running")
+ running = true
registerSources()
registerSinks()
sinks.foreach(_.start)
}
def stop() {
- sinks.foreach(_.stop)
+ if (running) {
+ sinks.foreach(_.stop)
+ } else {
+ logWarning("Stopping a MetricsSystem that is not running")
+ }
+ running = false
}
def report() {
@@ -107,7 +121,7 @@ private[spark] class MetricsSystem private (
* @return An unique metric name for each combination of
* application, executor/driver and metric source.
*/
- def buildRegistryName(source: Source): String = {
+ private[spark] def buildRegistryName(source: Source): String = {
val appId = conf.getOption("spark.app.id")
val executorId = conf.getOption("spark.executor.id")
val defaultName = MetricRegistry.name(source.sourceName)
@@ -144,7 +158,7 @@ private[spark] class MetricsSystem private (
})
}
- def registerSources() {
+ private def registerSources() {
val instConfig = metricsConfig.getInstance(instance)
val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX)
@@ -160,7 +174,7 @@ private[spark] class MetricsSystem private (
}
}
- def registerSinks() {
+ private def registerSinks() {
val instConfig = metricsConfig.getInstance(instance)
val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX)
diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
index 0027cbb0ff1fb..3f0950dae1f24 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
@@ -60,7 +60,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage
}
transportContext = new TransportContext(transportConf, rpcHandler)
clientFactory = transportContext.createClientFactory(bootstrap.toList)
- server = transportContext.createServer()
+ server = transportContext.createServer(conf.getInt("spark.blockManager.port", 0))
appId = conf.getAppId
logInfo("Server created on " + server.getPort)
}
diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala
index df4b085d2251e..3340fca08014e 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala
@@ -81,11 +81,24 @@ private[nio] class ConnectionManager(
private val ackTimeoutMonitor =
new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor"))
- private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60)
+ private val ackTimeout =
+ conf.getInt("spark.core.connection.ack.wait.timeout", conf.getInt("spark.network.timeout", 120))
+
+ // Get the thread counts from the Spark Configuration.
+ //
+ // Even though the ThreadPoolExecutor constructor takes both a minimum and maximum value,
+ // we only query for the minimum value because we are using LinkedBlockingDeque.
+ //
+ // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is
+ // an unbounded queue) no more than corePoolSize threads will ever be created, so only the "min"
+ // parameter is necessary.
+ private val handlerThreadCount = conf.getInt("spark.core.connection.handler.threads.min", 20)
+ private val ioThreadCount = conf.getInt("spark.core.connection.io.threads.min", 4)
+ private val connectThreadCount = conf.getInt("spark.core.connection.connect.threads.min", 1)
private val handleMessageExecutor = new ThreadPoolExecutor(
- conf.getInt("spark.core.connection.handler.threads.min", 20),
- conf.getInt("spark.core.connection.handler.threads.max", 60),
+ handlerThreadCount,
+ handlerThreadCount,
conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable](),
Utils.namedThreadFactory("handle-message-executor")) {
@@ -96,12 +109,11 @@ private[nio] class ConnectionManager(
logError("Error in handleMessageExecutor is not handled properly", t)
}
}
-
}
private val handleReadWriteExecutor = new ThreadPoolExecutor(
- conf.getInt("spark.core.connection.io.threads.min", 4),
- conf.getInt("spark.core.connection.io.threads.max", 32),
+ ioThreadCount,
+ ioThreadCount,
conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable](),
Utils.namedThreadFactory("handle-read-write-executor")) {
@@ -112,14 +124,13 @@ private[nio] class ConnectionManager(
logError("Error in handleReadWriteExecutor is not handled properly", t)
}
}
-
}
// Use a different, yet smaller, thread pool - infrequently used with very short lived tasks :
// which should be executed asap
private val handleConnectExecutor = new ThreadPoolExecutor(
- conf.getInt("spark.core.connection.connect.threads.min", 1),
- conf.getInt("spark.core.connection.connect.threads.max", 8),
+ connectThreadCount,
+ connectThreadCount,
conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable](),
Utils.namedThreadFactory("handle-connect-executor")) {
@@ -130,7 +141,6 @@ private[nio] class ConnectionManager(
logError("Error in handleConnectExecutor is not handled properly", t)
}
}
-
}
private val serverChannel = ServerSocketChannel.open()
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index ffc0a8a6d67eb..70edf191d928a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -60,7 +60,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]
* A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a
* tuple with the list of values for that key.
*
- * Note: This is an internal API. We recommend users use RDD.coGroup(...) instead of
+ * Note: This is an internal API. We recommend users use RDD.cogroup(...) instead of
* instantiating this directly.
* @param rdds parent RDDs.
@@ -70,8 +70,8 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]
class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner)
extends RDD[(K, Array[Iterable[_]])](rdds.head.context, Nil) {
- // For example, `(k, a) cogroup (k, b)` produces k -> Seq(ArrayBuffer as, ArrayBuffer bs).
- // Each ArrayBuffer is represented as a CoGroup, and the resulting Seq as a CoGroupCombiner.
+ // For example, `(k, a) cogroup (k, b)` produces k -> Array(ArrayBuffer as, ArrayBuffer bs).
+ // Each ArrayBuffer is represented as a CoGroup, and the resulting Array as a CoGroupCombiner.
// CoGroupValue is the intermediate state of each value before being merged in compute.
private type CoGroup = CompactBuffer[Any]
private type CoGroupValue = (Any, Int) // Int is dependency number
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 7792e8c53a819..fded29feb1bbe 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -131,7 +131,7 @@ class HadoopRDD[K, V](
// used to build JobTracker ID
private val createTime = new Date()
- private val shouldCloneJobConf = sc.conf.get("spark.hadoop.cloneConf", "false").toBoolean
+ private val shouldCloneJobConf = sc.conf.getBoolean("spark.hadoop.cloneConf", false)
// Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads.
protected def getJobConf(): JobConf = {
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index c767ce607943b..c53950b255629 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -25,6 +25,7 @@ import scala.collection.{Map, mutable}
import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.reflect.ClassTag
+import scala.util.DynamicVariable
import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus
import org.apache.hadoop.conf.{Configurable, Configuration}
@@ -84,7 +85,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
throw new SparkException("Default partitioner cannot partition array keys.")
}
}
- val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners)
+ val aggregator = new Aggregator[K, V, C](
+ self.context.clean(createCombiner),
+ self.context.clean(mergeValue),
+ self.context.clean(mergeCombiners))
if (self.partitioner == Some(partitioner)) {
self.mapPartitions(iter => {
val context = TaskContext.get()
@@ -120,11 +124,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
def aggregateByKey[U: ClassTag](zeroValue: U, partitioner: Partitioner)(seqOp: (U, V) => U,
combOp: (U, U) => U): RDD[(K, U)] = {
// Serialize the zero value to a byte array so that we can get a new clone of it on each key
- val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue)
+ val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue)
val zeroArray = new Array[Byte](zeroBuffer.limit)
zeroBuffer.get(zeroArray)
- lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance()
+ lazy val cachedSerializer = SparkEnv.get.serializer.newInstance()
val createZero = () => cachedSerializer.deserialize[U](ByteBuffer.wrap(zeroArray))
combineByKey[U]((v: V) => seqOp(createZero(), v), seqOp, combOp, partitioner)
@@ -165,12 +169,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
*/
def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = {
// Serialize the zero value to a byte array so that we can get a new clone of it on each key
- val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue)
+ val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue)
val zeroArray = new Array[Byte](zeroBuffer.limit)
zeroBuffer.get(zeroArray)
// When deserializing, use a lazy val to create just one instance of the serializer per task
- lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance()
+ lazy val cachedSerializer = SparkEnv.get.serializer.newInstance()
val createZero = () => cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray))
combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner)
@@ -433,6 +437,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Note: This operation may be very expensive. If you are grouping in order to perform an
* aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
* or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
+ *
+ * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any
+ * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
*/
def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = {
// groupByKey shouldn't use map side combine because map side combine does not
@@ -454,6 +461,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Note: This operation may be very expensive. If you are grouping in order to perform an
* aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
* or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
+ *
+ * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any
+ * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
*/
def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = {
groupByKey(new HashPartitioner(numPartitions))
@@ -480,7 +490,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
*/
def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = {
this.cogroup(other, partitioner).flatMapValues( pair =>
- for (v <- pair._1; w <- pair._2) yield (v, w)
+ for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, w)
)
}
@@ -493,9 +503,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = {
this.cogroup(other, partitioner).flatMapValues { pair =>
if (pair._2.isEmpty) {
- pair._1.map(v => (v, None))
+ pair._1.iterator.map(v => (v, None))
} else {
- for (v <- pair._1; w <- pair._2) yield (v, Some(w))
+ for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, Some(w))
}
}
}
@@ -510,9 +520,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
: RDD[(K, (Option[V], W))] = {
this.cogroup(other, partitioner).flatMapValues { pair =>
if (pair._1.isEmpty) {
- pair._2.map(w => (None, w))
+ pair._2.iterator.map(w => (None, w))
} else {
- for (v <- pair._1; w <- pair._2) yield (Some(v), w)
+ for (v <- pair._1.iterator; w <- pair._2.iterator) yield (Some(v), w)
}
}
}
@@ -528,9 +538,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
def fullOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner)
: RDD[(K, (Option[V], Option[W]))] = {
this.cogroup(other, partitioner).flatMapValues {
- case (vs, Seq()) => vs.map(v => (Some(v), None))
- case (Seq(), ws) => ws.map(w => (None, Some(w)))
- case (vs, ws) => for (v <- vs; w <- ws) yield (Some(v), Some(w))
+ case (vs, Seq()) => vs.iterator.map(v => (Some(v), None))
+ case (Seq(), ws) => ws.iterator.map(w => (None, Some(w)))
+ case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), Some(w))
}
}
@@ -961,7 +971,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
val outfmt = job.getOutputFormatClass
val jobFormat = outfmt.newInstance
- if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) {
+ if (isOutputSpecValidationEnabled) {
// FileOutputFormat ignores the filesystem parameter
jobFormat.checkOutputSpecs(job)
}
@@ -1039,7 +1049,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " +
valueClass.getSimpleName + ")")
- if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) {
+ if (isOutputSpecValidationEnabled) {
// FileOutputFormat ignores the filesystem parameter
val ignoredFs = FileSystem.get(hadoopConf)
hadoopConf.getOutputFormat.checkOutputSpecs(ignoredFs, hadoopConf)
@@ -1114,8 +1124,22 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
private[spark] def valueClass: Class[_] = vt.runtimeClass
private[spark] def keyOrdering: Option[Ordering[K]] = Option(ord)
+
+ // Note: this needs to be a function instead of a 'val' so that the disableOutputSpecValidation
+ // setting can take effect:
+ private def isOutputSpecValidationEnabled: Boolean = {
+ val validationDisabled = PairRDDFunctions.disableOutputSpecValidation.value
+ val enabledInConf = self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)
+ enabledInConf && !validationDisabled
+ }
}
private[spark] object PairRDDFunctions {
val RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES = 256
+
+ /**
+ * Allows for the `spark.hadoop.validateOutputSpecs` checks to be disabled on a case-by-case
+ * basis; see SPARK-4835 for more details.
+ */
+ val disableOutputSpecValidation: DynamicVariable[Boolean] = new DynamicVariable[Boolean](false)
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 214f22bc5b603..5118e2b911120 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -1146,15 +1146,20 @@ abstract class RDD[T: ClassTag](
if (num == 0) {
Array.empty
} else {
- mapPartitions { items =>
+ val mapRDDs = mapPartitions { items =>
// Priority keeps the largest elements, so let's reverse the ordering.
val queue = new BoundedPriorityQueue[T](num)(ord.reverse)
queue ++= util.collection.Utils.takeOrdered(items, num)(ord)
Iterator.single(queue)
- }.reduce { (queue1, queue2) =>
- queue1 ++= queue2
- queue1
- }.toArray.sorted(ord)
+ }
+ if (mapRDDs.partitions.size == 0) {
+ Array.empty
+ } else {
+ mapRDDs.reduce { (queue1, queue2) =>
+ queue1 ++= queue2
+ queue1
+ }.toArray.sorted(ord)
+ }
}
}
@@ -1174,7 +1179,26 @@ abstract class RDD[T: ClassTag](
* Save this RDD as a text file, using string representations of elements.
*/
def saveAsTextFile(path: String) {
- this.map(x => (NullWritable.get(), new Text(x.toString)))
+ // https://issues.apache.org/jira/browse/SPARK-2075
+ //
+ // NullWritable is a `Comparable` in Hadoop 1.+, so the compiler cannot find an implicit
+ // Ordering for it and will use the default `null`. However, it's a `Comparable[NullWritable]`
+ // in Hadoop 2.+, so the compiler will call the implicit `Ordering.ordered` method to create an
+ // Ordering for `NullWritable`. That's why the compiler will generate different anonymous
+ // classes for `saveAsTextFile` in Hadoop 1.+ and Hadoop 2.+.
+ //
+ // Therefore, here we provide an explicit Ordering `null` to make sure the compiler generate
+ // same bytecodes for `saveAsTextFile`.
+ val nullWritableClassTag = implicitly[ClassTag[NullWritable]]
+ val textClassTag = implicitly[ClassTag[Text]]
+ val r = this.mapPartitions { iter =>
+ val text = new Text()
+ iter.map { x =>
+ text.set(x.toString)
+ (NullWritable.get(), text)
+ }
+ }
+ RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null)
.saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path)
}
@@ -1182,7 +1206,17 @@ abstract class RDD[T: ClassTag](
* Save this RDD as a compressed text file, using string representations of elements.
*/
def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) {
- this.map(x => (NullWritable.get(), new Text(x.toString)))
+ // https://issues.apache.org/jira/browse/SPARK-2075
+ val nullWritableClassTag = implicitly[ClassTag[NullWritable]]
+ val textClassTag = implicitly[ClassTag[Text]]
+ val r = this.mapPartitions { iter =>
+ val text = new Text()
+ iter.map { x =>
+ text.set(x.toString)
+ (NullWritable.get(), text)
+ }
+ }
+ RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null)
.saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index cb8ccfbdbdcbb..259621d263d7c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -138,6 +138,7 @@ class DAGScheduler(
}
initializeEventProcessActor()
+ taskScheduler.setDAGScheduler(this)
// Called by TaskScheduler to report task's starting.
def taskStarted(task: Task[_], taskInfo: TaskInfo) {
@@ -1375,12 +1376,6 @@ private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler)
private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGScheduler)
extends Actor with Logging {
- override def preStart() {
- // set DAGScheduler for taskScheduler to ensure eventProcessActor is always
- // valid when the messages arrive
- dagScheduler.taskScheduler.setDAGScheduler(dagScheduler)
- }
-
/**
* The main event loop of the DAG scheduler.
*/
diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
index 597dbc884913c..27bf4f1599076 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
@@ -17,20 +17,23 @@
package org.apache.spark.scheduler
+import java.io._
+import java.net.URI
+
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
+import com.google.common.base.Charsets
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
import org.apache.hadoop.fs.permission.FsPermission
import org.json4s.JsonAST.JValue
import org.json4s.jackson.JsonMethods._
-import org.apache.spark.{Logging, SparkConf, SparkContext}
+import org.apache.spark.{Logging, SparkConf, SPARK_VERSION}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.io.CompressionCodec
-import org.apache.spark.SPARK_VERSION
-import org.apache.spark.util.{FileLogger, JsonProtocol, Utils}
+import org.apache.spark.util.{JsonProtocol, Utils}
/**
* A SparkListener that logs events to persistent storage.
@@ -58,36 +61,78 @@ private[spark] class EventLoggingListener(
private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false)
private val testing = sparkConf.getBoolean("spark.eventLog.testing", false)
private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024
- val logDir = EventLoggingListener.getLogDirPath(logBaseDir, appId)
- val logDirName: String = logDir.split("/").last
- protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize,
- shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS))
+ private val fileSystem = Utils.getHadoopFileSystem(new URI(logBaseDir), hadoopConf)
+
+ // Only defined if the file system scheme is not local
+ private var hadoopDataStream: Option[FSDataOutputStream] = None
+
+ // The Hadoop APIs have changed over time, so we use reflection to figure out
+ // the correct method to use to flush a hadoop data stream. See SPARK-1518
+ // for details.
+ private val hadoopFlushMethod = {
+ val cls = classOf[FSDataOutputStream]
+ scala.util.Try(cls.getMethod("hflush")).getOrElse(cls.getMethod("sync"))
+ }
+
+ private var writer: Option[PrintWriter] = None
// For testing. Keep track of all JSON serialized events that have been logged.
private[scheduler] val loggedEvents = new ArrayBuffer[JValue]
+ // Visible for tests only.
+ private[scheduler] val logPath = getLogPath(logBaseDir, appId)
+
/**
- * Begin logging events.
- * If compression is used, log a file that indicates which compression library is used.
+ * Creates the log file in the configured log directory.
*/
def start() {
- logger.start()
- logInfo("Logging events to %s".format(logDir))
- if (shouldCompress) {
- val codec =
- sparkConf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)
- logger.newFile(COMPRESSION_CODEC_PREFIX + codec)
+ if (!fileSystem.isDirectory(new Path(logBaseDir))) {
+ throw new IllegalArgumentException(s"Log directory $logBaseDir does not exist.")
+ }
+
+ val workingPath = logPath + IN_PROGRESS
+ val uri = new URI(workingPath)
+ val path = new Path(workingPath)
+ val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme
+ val isDefaultLocal = defaultFs == null || defaultFs == "file"
+
+ if (shouldOverwrite && fileSystem.exists(path)) {
+ logWarning(s"Event log $path already exists. Overwriting...")
+ fileSystem.delete(path, true)
}
- logger.newFile(SPARK_VERSION_PREFIX + SPARK_VERSION)
- logger.newFile(LOG_PREFIX + logger.fileIndex)
+
+ /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844).
+ * Therefore, for local files, use FileOutputStream instead. */
+ val dstream =
+ if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") {
+ new FileOutputStream(uri.getPath)
+ } else {
+ hadoopDataStream = Some(fileSystem.create(path))
+ hadoopDataStream.get
+ }
+
+ val compressionCodec =
+ if (shouldCompress) {
+ Some(CompressionCodec.createCodec(sparkConf))
+ } else {
+ None
+ }
+
+ fileSystem.setPermission(path, LOG_FILE_PERMISSIONS)
+ val logStream = initEventLog(new BufferedOutputStream(dstream, outputBufferSize),
+ compressionCodec)
+ writer = Some(new PrintWriter(logStream))
+
+ logInfo("Logging events to %s".format(logPath))
}
/** Log the event as JSON. */
private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) {
val eventJson = JsonProtocol.sparkEventToJson(event)
- logger.logLine(compact(render(eventJson)))
+ writer.foreach(_.println(compact(render(eventJson))))
if (flushLogger) {
- logger.flush()
+ writer.foreach(_.flush())
+ hadoopDataStream.foreach(hadoopFlushMethod.invoke(_))
}
if (testing) {
loggedEvents += eventJson
@@ -123,130 +168,164 @@ private[spark] class EventLoggingListener(
logEvent(event, flushLogger = true)
override def onApplicationEnd(event: SparkListenerApplicationEnd) =
logEvent(event, flushLogger = true)
+
// No-op because logging every update would be overkill
override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { }
/**
- * Stop logging events.
- * In addition, create an empty special file to indicate application completion.
+ * Stop logging events. The event log file will be renamed so that it loses the
+ * ".inprogress" suffix.
*/
def stop() = {
- logger.newFile(APPLICATION_COMPLETE)
- logger.stop()
+ writer.foreach(_.close())
+
+ val target = new Path(logPath)
+ if (fileSystem.exists(target)) {
+ if (shouldOverwrite) {
+ logWarning(s"Event log $target already exists. Overwriting...")
+ fileSystem.delete(target, true)
+ } else {
+ throw new IOException("Target log file already exists (%s)".format(logPath))
+ }
+ }
+ fileSystem.rename(new Path(logPath + IN_PROGRESS), target)
}
+
}
private[spark] object EventLoggingListener extends Logging {
+ // Suffix applied to the names of files still being written by applications.
+ val IN_PROGRESS = ".inprogress"
val DEFAULT_LOG_DIR = "/tmp/spark-events"
- val LOG_PREFIX = "EVENT_LOG_"
- val SPARK_VERSION_PREFIX = "SPARK_VERSION_"
- val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_"
- val APPLICATION_COMPLETE = "APPLICATION_COMPLETE"
- val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
- // A cache for compression codecs to avoid creating the same codec many times
- private val codecMap = new mutable.HashMap[String, CompressionCodec]
+ private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
- def isEventLogFile(fileName: String): Boolean = {
- fileName.startsWith(LOG_PREFIX)
- }
+ // Marker for the end of header data in a log file. After this marker, log data, potentially
+ // compressed, will be found.
+ private val HEADER_END_MARKER = "=== LOG_HEADER_END ==="
- def isSparkVersionFile(fileName: String): Boolean = {
- fileName.startsWith(SPARK_VERSION_PREFIX)
- }
+ // To avoid corrupted files causing the heap to fill up. Value is arbitrary.
+ private val MAX_HEADER_LINE_LENGTH = 4096
- def isCompressionCodecFile(fileName: String): Boolean = {
- fileName.startsWith(COMPRESSION_CODEC_PREFIX)
- }
+ // A cache for compression codecs to avoid creating the same codec many times
+ private val codecMap = new mutable.HashMap[String, CompressionCodec]
- def isApplicationCompleteFile(fileName: String): Boolean = {
- fileName == APPLICATION_COMPLETE
- }
+ /**
+ * Write metadata about the event log to the given stream.
+ *
+ * The header is a serialized version of a map, except it does not use Java serialization to
+ * avoid incompatibilities between different JDKs. It writes one map entry per line, in
+ * "key=value" format.
+ *
+ * The very last entry in the header is the `HEADER_END_MARKER` marker, so that the parsing code
+ * can know when to stop.
+ *
+ * The format needs to be kept in sync with the openEventLog() method below. Also, it cannot
+ * change in new Spark versions without some other way of detecting the change (like some
+ * metadata encoded in the file name).
+ *
+ * @param logStream Raw output stream to the even log file.
+ * @param compressionCodec Optional compression codec to use.
+ * @return A stream where to write event log data. This may be a wrapper around the original
+ * stream (for example, when compression is enabled).
+ */
+ def initEventLog(
+ logStream: OutputStream,
+ compressionCodec: Option[CompressionCodec]): OutputStream = {
+ val meta = mutable.HashMap(("version" -> SPARK_VERSION))
+ compressionCodec.foreach { codec =>
+ meta += ("compressionCodec" -> codec.getClass().getName())
+ }
- def parseSparkVersion(fileName: String): String = {
- if (isSparkVersionFile(fileName)) {
- fileName.replaceAll(SPARK_VERSION_PREFIX, "")
- } else ""
- }
+ def write(entry: String) = {
+ val bytes = entry.getBytes(Charsets.UTF_8)
+ if (bytes.length > MAX_HEADER_LINE_LENGTH) {
+ throw new IOException(s"Header entry too long: ${entry}")
+ }
+ logStream.write(bytes, 0, bytes.length)
+ }
- def parseCompressionCodec(fileName: String): String = {
- if (isCompressionCodecFile(fileName)) {
- fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "")
- } else ""
+ meta.foreach { case (k, v) => write(s"$k=$v\n") }
+ write(s"$HEADER_END_MARKER\n")
+ compressionCodec.map(_.compressedOutputStream(logStream)).getOrElse(logStream)
}
/**
- * Return a file-system-safe path to the log directory for the given application.
+ * Return a file-system-safe path to the log file for the given application.
*
- * @param logBaseDir A base directory for the path to the log directory for given application.
+ * @param logBaseDir Directory where the log file will be written.
* @param appId A unique app ID.
* @return A path which consists of file-system-safe characters.
*/
- def getLogDirPath(logBaseDir: String, appId: String): String = {
+ def getLogPath(logBaseDir: String, appId: String): String = {
val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase
Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/")
}
/**
- * Parse the event logging information associated with the logs in the given directory.
+ * Opens an event log file and returns an input stream to the event data.
*
- * Specifically, this looks for event log files, the Spark version file, the compression
- * codec file (if event logs are compressed), and the application completion file (if the
- * application has run to completion).
+ * @return 2-tuple (event input stream, Spark version of event data)
*/
- def parseLoggingInfo(logDir: Path, fileSystem: FileSystem): EventLoggingInfo = {
+ def openEventLog(log: Path, fs: FileSystem): (InputStream, String) = {
+ // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain
+ // IOException when a file does not exist, so try our best to throw a proper exception.
+ if (!fs.exists(log)) {
+ throw new FileNotFoundException(s"File $log does not exist.")
+ }
+
+ val in = new BufferedInputStream(fs.open(log))
+ // Read a single line from the input stream without buffering.
+ // We cannot use BufferedReader because we must avoid reading
+ // beyond the end of the header, after which the content of the
+ // file may be compressed.
+ def readLine(): String = {
+ val bytes = new ByteArrayOutputStream()
+ var next = in.read()
+ var count = 0
+ while (next != '\n') {
+ if (next == -1) {
+ throw new IOException("Unexpected end of file.")
+ }
+ bytes.write(next)
+ count = count + 1
+ if (count > MAX_HEADER_LINE_LENGTH) {
+ throw new IOException("Maximum header line length exceeded.")
+ }
+ next = in.read()
+ }
+ new String(bytes.toByteArray(), Charsets.UTF_8)
+ }
+
+ // Parse the header metadata in the form of k=v pairs
+ // This assumes that every line before the header end marker follows this format
try {
- val fileStatuses = fileSystem.listStatus(logDir)
- val filePaths =
- if (fileStatuses != null) {
- fileStatuses.filter(!_.isDir).map(_.getPath).toSeq
- } else {
- Seq[Path]()
+ val meta = new mutable.HashMap[String, String]()
+ var foundEndMarker = false
+ while (!foundEndMarker) {
+ readLine() match {
+ case HEADER_END_MARKER =>
+ foundEndMarker = true
+ case entry =>
+ val prop = entry.split("=", 2)
+ if (prop.length != 2) {
+ throw new IllegalArgumentException("Invalid metadata in log file.")
+ }
+ meta += (prop(0) -> prop(1))
}
- if (filePaths.isEmpty) {
- logWarning("No files found in logging directory %s".format(logDir))
}
- EventLoggingInfo(
- logPaths = filePaths.filter { path => isEventLogFile(path.getName) },
- sparkVersion = filePaths
- .find { path => isSparkVersionFile(path.getName) }
- .map { path => parseSparkVersion(path.getName) }
- .getOrElse(""),
- compressionCodec = filePaths
- .find { path => isCompressionCodecFile(path.getName) }
- .map { path =>
- val codec = EventLoggingListener.parseCompressionCodec(path.getName)
- val conf = new SparkConf
- conf.set("spark.io.compression.codec", codec)
- codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf))
- },
- applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) }
- )
+
+ val sparkVersion = meta.get("version").getOrElse(
+ throw new IllegalArgumentException("Missing Spark version in log metadata."))
+ val codec = meta.get("compressionCodec").map { codecName =>
+ codecMap.getOrElseUpdate(codecName, CompressionCodec.createCodec(new SparkConf, codecName))
+ }
+ (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion)
} catch {
case e: Exception =>
- logError("Exception in parsing logging info from directory %s".format(logDir), e)
- EventLoggingInfo.empty
+ in.close()
+ throw e
}
}
- /**
- * Parse the event logging information associated with the logs in the given directory.
- */
- def parseLoggingInfo(logDir: String, fileSystem: FileSystem): EventLoggingInfo = {
- parseLoggingInfo(new Path(logDir), fileSystem)
- }
-}
-
-
-/**
- * Information needed to process the event logs associated with an application.
- */
-private[spark] case class EventLoggingInfo(
- logPaths: Seq[Path],
- sparkVersion: String,
- compressionCodec: Option[CompressionCodec],
- applicationComplete: Boolean = false)
-
-private[spark] object EventLoggingInfo {
- def empty = EventLoggingInfo(Seq[Path](), "", None, applicationComplete = false)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
index 01d5943d777f3..1efce124c0a6b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
@@ -122,7 +122,7 @@ private[spark] class CompressedMapStatus(
/**
* A [[MapStatus]] implementation that only stores the average size of non-empty blocks,
- * plus a bitmap for tracking which blocks are non-empty. During serialization, this bitmap
+ * plus a bitmap for tracking which blocks are empty. During serialization, this bitmap
* is compressed.
*
* @param loc location where the task is being executed
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
index f89724d4ea196..584f4e7789d1a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
@@ -17,74 +17,45 @@
package org.apache.spark.scheduler
-import java.io.{BufferedInputStream, InputStream}
+import java.io.{InputStream, IOException}
import scala.io.Source
-import org.apache.hadoop.fs.{Path, FileSystem}
import org.json4s.jackson.JsonMethods._
import org.apache.spark.Logging
-import org.apache.spark.io.CompressionCodec
import org.apache.spark.util.JsonProtocol
/**
- * A SparkListenerBus that replays logged events from persisted storage.
- *
- * This assumes the given paths are valid log files, where each line can be deserialized into
- * exactly one SparkListenerEvent.
+ * A SparkListenerBus that can be used to replay events from serialized event data.
*/
-private[spark] class ReplayListenerBus(
- logPaths: Seq[Path],
- fileSystem: FileSystem,
- compressionCodec: Option[CompressionCodec])
- extends SparkListenerBus with Logging {
-
- private var replayed = false
-
- if (logPaths.length == 0) {
- logWarning("Log path provided contains no log files.")
- }
+private[spark] class ReplayListenerBus extends SparkListenerBus with Logging {
/**
- * Replay each event in the order maintained in the given logs.
- * This should only be called exactly once.
+ * Replay each event in the order maintained in the given stream. The stream is expected to
+ * contain one JSON-encoded SparkListenerEvent per line.
+ *
+ * This method can be called multiple times, but the listener behavior is undefined after any
+ * error is thrown by this method.
+ *
+ * @param logData Stream containing event log data.
+ * @param version Spark version that generated the events.
*/
- def replay() {
- assert(!replayed, "ReplayListenerBus cannot replay events more than once")
- logPaths.foreach { path =>
- // Keep track of input streams at all levels to close them later
- // This is necessary because an exception can occur in between stream initializations
- var fileStream: Option[InputStream] = None
- var bufferedStream: Option[InputStream] = None
- var compressStream: Option[InputStream] = None
- var currentLine = ""
- try {
- fileStream = Some(fileSystem.open(path))
- bufferedStream = Some(new BufferedInputStream(fileStream.get))
- compressStream = Some(wrapForCompression(bufferedStream.get))
-
- // Parse each line as an event and post the event to all attached listeners
- val lines = Source.fromInputStream(compressStream.get).getLines()
- lines.foreach { line =>
- currentLine = line
- postToAll(JsonProtocol.sparkEventFromJson(parse(line)))
- }
- } catch {
- case e: Exception =>
- logError("Exception in parsing Spark event log %s".format(path), e)
- logError("Malformed line: %s\n".format(currentLine))
- } finally {
- fileStream.foreach(_.close())
- bufferedStream.foreach(_.close())
- compressStream.foreach(_.close())
+ def replay(logData: InputStream, version: String) {
+ var currentLine: String = null
+ try {
+ val lines = Source.fromInputStream(logData).getLines()
+ lines.foreach { line =>
+ currentLine = line
+ postToAll(JsonProtocol.sparkEventFromJson(parse(line)))
}
+ } catch {
+ case ioe: IOException =>
+ throw ioe
+ case e: Exception =>
+ logError("Exception in parsing Spark event log.", e)
+ logError("Malformed line: %s\n".format(currentLine))
}
- replayed = true
}
- /** If a compression codec is specified, wrap the given stream in a compression stream. */
- private def wrapForCompression(stream: InputStream): InputStream = {
- compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream)
- }
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index 1eb2739982523..1727d69e3c736 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -45,7 +45,7 @@ import org.apache.spark.util.Utils
private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable {
final def run(attemptId: Long): T = {
- context = new TaskContextImpl(stageId, partitionId, attemptId, false)
+ context = new TaskContextImpl(stageId, partitionId, attemptId, runningLocally = false)
TaskContextHelper.setTaskContext(context)
context.taskMetrics.setHostname(Utils.localHostName())
taskThread = Thread.currentThread()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
index c4f16c783a40a..9fa5a09cc29d7 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
@@ -19,6 +19,7 @@ package org.apache.spark.scheduler
import java.nio.ByteBuffer
+import scala.language.existentials
import scala.util.control.NonFatal
import org.apache.spark._
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index cd3c015321e85..a41f3eef195d2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -394,9 +394,6 @@ private[spark] class TaskSchedulerImpl(
taskResultGetter.stop()
}
starvationTimer.cancel()
-
- // sleeping for an arbitrary 1 seconds to ensure that messages are sent out.
- Thread.sleep(1000L)
}
override def defaultParallelism() = backend.defaultParallelism()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index cabdc655f89bf..28e6147509f78 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -249,7 +249,7 @@ private[spark] class TaskSetManager(
* 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(execId: String, list: ArrayBuffer[Int]): Option[Int] = {
+ private def dequeueTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = {
var indexOffset = list.size
while (indexOffset > 0) {
indexOffset -= 1
@@ -290,7 +290,7 @@ private[spark] class TaskSetManager(
* 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)
+ private def dequeueSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value)
: Option[(Int, TaskLocality.Value)] =
{
speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set
@@ -366,22 +366,22 @@ private[spark] class TaskSetManager(
*
* @return An option containing (task index within the task set, locality, is speculative?)
*/
- private def findTask(execId: String, host: String, maxLocality: TaskLocality.Value)
+ private def dequeueTask(execId: String, host: String, maxLocality: TaskLocality.Value)
: Option[(Int, TaskLocality.Value, Boolean)] =
{
- for (index <- findTaskFromList(execId, getPendingTasksForExecutor(execId))) {
+ for (index <- dequeueTaskFromList(execId, getPendingTasksForExecutor(execId))) {
return Some((index, TaskLocality.PROCESS_LOCAL, false))
}
if (TaskLocality.isAllowed(maxLocality, TaskLocality.NODE_LOCAL)) {
- for (index <- findTaskFromList(execId, getPendingTasksForHost(host))) {
+ for (index <- dequeueTaskFromList(execId, getPendingTasksForHost(host))) {
return Some((index, TaskLocality.NODE_LOCAL, false))
}
}
if (TaskLocality.isAllowed(maxLocality, TaskLocality.NO_PREF)) {
// Look for noPref tasks after NODE_LOCAL for minimize cross-rack traffic
- for (index <- findTaskFromList(execId, pendingTasksWithNoPrefs)) {
+ for (index <- dequeueTaskFromList(execId, pendingTasksWithNoPrefs)) {
return Some((index, TaskLocality.PROCESS_LOCAL, false))
}
}
@@ -389,20 +389,20 @@ private[spark] class TaskSetManager(
if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) {
for {
rack <- sched.getRackForHost(host)
- index <- findTaskFromList(execId, getPendingTasksForRack(rack))
+ index <- dequeueTaskFromList(execId, getPendingTasksForRack(rack))
} {
return Some((index, TaskLocality.RACK_LOCAL, false))
}
}
if (TaskLocality.isAllowed(maxLocality, TaskLocality.ANY)) {
- for (index <- findTaskFromList(execId, allPendingTasks)) {
+ for (index <- dequeueTaskFromList(execId, allPendingTasks)) {
return Some((index, TaskLocality.ANY, false))
}
}
// find a speculative task if all others tasks have been scheduled
- findSpeculativeTask(execId, host, maxLocality).map {
+ dequeueSpeculativeTask(execId, host, maxLocality).map {
case (taskIndex, allowedLocality) => (taskIndex, allowedLocality, true)}
}
@@ -436,7 +436,7 @@ private[spark] class TaskSetManager(
}
}
- findTask(execId, host, allowedLocality) match {
+ dequeueTask(execId, host, allowedLocality) match {
case Some((index, taskLocality, speculative)) => {
// Found a task; do some bookkeeping and return a task description
val task = tasks(index)
@@ -704,7 +704,7 @@ private[spark] class TaskSetManager(
// Re-enqueue pending tasks for this host based on the status of the cluster. 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.
+ // locations), because dequeueTaskFromList will skip already-running tasks.
for (index <- getPendingTasksForExecutor(execId)) {
addPendingTask(index, readding=true)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 29cd34429b881..fe9914b50bc54 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -27,7 +27,7 @@ import akka.actor._
import akka.pattern.ask
import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
-import org.apache.spark.{SparkEnv, Logging, SparkException, TaskState}
+import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState}
import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer}
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils}
@@ -42,7 +42,7 @@ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Ut
*/
private[spark]
class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSystem: ActorSystem)
- extends SchedulerBackend with Logging
+ extends ExecutorAllocationClient with 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)
@@ -307,7 +307,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste
* Request an additional number of executors from the cluster manager.
* Return whether the request is acknowledged.
*/
- final def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized {
+ final override def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized {
logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager")
logDebug(s"Number of pending executors is now $numPendingExecutors")
numPendingExecutors += numAdditionalExecutors
@@ -334,7 +334,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste
* Request that the cluster manager kill the specified executors.
* Return whether the kill request is acknowledged.
*/
- final def killExecutors(executorIds: Seq[String]): Boolean = synchronized {
+ final override def killExecutors(executorIds: Seq[String]): Boolean = synchronized {
logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}")
val filteredExecutorIds = new ArrayBuffer[String]
executorIds.foreach { id =>
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 8c7de75600b5f..7eb87a564d6f5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -55,19 +55,26 @@ private[spark] class SparkDeploySchedulerBackend(
"{{WORKER_URL}}")
val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions")
.map(Utils.splitCommandString).getOrElse(Seq.empty)
- val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath").toSeq.flatMap { cp =>
- cp.split(java.io.File.pathSeparator)
- }
- val libraryPathEntries =
- sc.conf.getOption("spark.executor.extraLibraryPath").toSeq.flatMap { cp =>
- cp.split(java.io.File.pathSeparator)
+ val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath")
+ .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil)
+ val libraryPathEntries = sc.conf.getOption("spark.executor.extraLibraryPath")
+ .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil)
+
+ // When testing, expose the parent class path to the child. This is processed by
+ // compute-classpath.{cmd,sh} and makes all needed jars available to child processes
+ // when the assembly is built with the "*-provided" profiles enabled.
+ val testingClassPath =
+ if (sys.props.contains("spark.testing")) {
+ sys.props("java.class.path").split(java.io.File.pathSeparator).toSeq
+ } else {
+ Nil
}
// Start executors with a few necessary configs for registering with the scheduler
val sparkJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf)
val javaOpts = sparkJavaOpts ++ extraJavaOpts
val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend",
- args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts)
+ args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts)
val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("")
val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command,
appUIAddress, sc.eventLogDir)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
index a2f1f14264a99..b3bd3110ac809 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
@@ -41,17 +41,18 @@ private case class StopExecutor()
* and the TaskSchedulerImpl.
*/
private[spark] class LocalActor(
- scheduler: TaskSchedulerImpl,
- executorBackend: LocalBackend,
- private val totalCores: Int) extends Actor with ActorLogReceive with Logging {
+ scheduler: TaskSchedulerImpl,
+ executorBackend: LocalBackend,
+ private val totalCores: Int)
+ extends Actor with ActorLogReceive with Logging {
private var freeCores = totalCores
private val localExecutorId = SparkContext.DRIVER_IDENTIFIER
private val localExecutorHostname = "localhost"
- val executor = new Executor(
- localExecutorId, localExecutorHostname, scheduler.conf.getAll, totalCores, isLocal = true)
+ private val executor = new Executor(
+ localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true)
override def receiveWithLogging = {
case ReviveOffers =>
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 621a951c27d07..d2947dcea4f7c 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -26,6 +26,7 @@ import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializ
import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator}
import org.apache.spark._
+import org.apache.spark.api.python.PythonBroadcast
import org.apache.spark.broadcast.HttpBroadcast
import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock}
import org.apache.spark.scheduler.MapStatus
@@ -90,6 +91,7 @@ class KryoSerializer(conf: SparkConf)
// Allow sending SerializableWritable
kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer())
kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer())
+ kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer())
try {
// Use the default classloader when calling the user registrator.
diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala
index 801ae54086053..a44a8e1249256 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala
@@ -20,8 +20,8 @@ package org.apache.spark.shuffle
import org.apache.spark.{TaskContext, ShuffleDependency}
/**
- * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on both the
- * driver and executors, based on the spark.shuffle.manager setting. The driver registers shuffles
+ * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on the driver
+ * and on each executor, based on the spark.shuffle.manager setting. The driver registers shuffles
* with it, and executors (or tasks running locally in the driver) can ask to read and write data.
*
* NOTE: this will be instantiated by SparkEnv so its constructor can take a SparkConf and
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
index 9cbda41223a8b..64133464d8daa 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -52,8 +52,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
private val akkaTimeout = AkkaUtils.askTimeout(conf)
- val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs",
- math.max(conf.getInt("spark.executor.heartbeatInterval", 10000) * 3, 45000))
+ val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120 * 1000)
val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000)
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
index 58fba54710510..af05eb3ca69ce 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
@@ -17,7 +17,7 @@
package org.apache.spark.storage
-import java.io.File
+import java.io.{IOException, File}
import java.text.SimpleDateFormat
import java.util.{Date, Random, UUID}
@@ -71,7 +71,9 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon
old
} else {
val newDir = new File(localDirs(dirId), "%02x".format(subDirId))
- newDir.mkdir()
+ if (!newDir.exists() && !newDir.mkdir()) {
+ throw new IOException(s"Failed to create local dir in $newDir.")
+ }
subDirs(dirId)(subDirId) = newDir
newDir
}
@@ -164,7 +166,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon
/** Cleanup local dirs and stop shuffle sender. */
private[spark] def stop() {
// Only perform cleanup if an external service is not serving our shuffle files.
- if (!blockManager.externalShuffleServiceEnabled) {
+ if (!blockManager.externalShuffleServiceEnabled || blockManager.blockManagerId.isDriver) {
localDirs.foreach { localDir =>
if (localDir.isDirectory() && localDir.exists()) {
try {
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index 176907dffa46a..0c24ad2760e08 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -57,8 +57,6 @@ private[spark] class SparkUI private (
attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath))
attachHandler(
createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest))
- // If the UI is live, then serve
- sc.foreach { _.env.metricsSystem.getServletHandlers.foreach(attachHandler) }
}
initialize()
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index d970fa30c1c35..b5022fe853c49 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -194,9 +194,12 @@ private[spark] object UIUtils extends Logging {
-
-
-
+
{shortAppName} application UI
@@ -214,11 +217,6 @@ private[spark] object UIUtils extends Logging {
{content}
-