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 { - } @@ -236,8 +234,9 @@ private[spark] object UIUtils extends Logging {

- + + {org.apache.spark.SPARK_VERSION} {title}

@@ -245,11 +244,6 @@ private[spark] object UIUtils extends Logging {
{content} - } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 8c2457f56bffe..db2531dc171f8 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -65,7 +65,7 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeout = conf.getInt("spark.akka.timeout", 100) + val akkaTimeout = conf.getInt("spark.akka.timeout", conf.getInt("spark.network.timeout", 120)) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" @@ -89,7 +89,7 @@ private[spark] object AkkaUtils extends Logging { } val requireCookie = if (isAuthOn) "on" else "off" val secureCookie = if (isAuthOn) secretKey else "" - logDebug("In createActorSystem, requireCookie is: " + requireCookie) + logDebug(s"In createActorSystem, requireCookie is: $requireCookie") val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( ConfigFactory.parseString( @@ -140,8 +140,8 @@ private[spark] object AkkaUtils extends Logging { def maxFrameSizeBytes(conf: SparkConf): Int = { val frameSizeInMB = conf.getInt("spark.akka.frameSize", 10) if (frameSizeInMB > AKKA_MAX_FRAME_SIZE_IN_MB) { - throw new IllegalArgumentException("spark.akka.frameSize should not be greater than " - + AKKA_MAX_FRAME_SIZE_IN_MB + "MB") + throw new IllegalArgumentException( + s"spark.akka.frameSize should not be greater than $AKKA_MAX_FRAME_SIZE_IN_MB MB") } frameSizeInMB * 1024 * 1024 } @@ -182,8 +182,8 @@ private[spark] object AkkaUtils extends Logging { timeout: FiniteDuration): T = { // TODO: Consider removing multiple attempts if (actor == null) { - throw new SparkException("Error sending message as actor is null " + - "[message = " + message + "]") + throw new SparkException(s"Error sending message [message = $message]" + + " as actor is null ") } var attempts = 0 var lastException: Exception = null @@ -200,13 +200,13 @@ private[spark] object AkkaUtils extends Logging { case ie: InterruptedException => throw ie case e: Exception => lastException = e - logWarning("Error sending message in " + attempts + " attempts", e) + logWarning(s"Error sending message [message = $message] in $attempts attempts", e) } Thread.sleep(retryInterval) } throw new SparkException( - "Error sending message [message = " + message + "]", lastException) + s"Error sending message [message = $message]", lastException) } def makeDriverRef(name: String, conf: SparkConf, actorSystem: ActorSystem): ActorRef = { diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala deleted file mode 100644 index fdc73f08261a6..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ /dev/null @@ -1,237 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import java.io.{BufferedOutputStream, FileOutputStream, IOException, PrintWriter} -import java.net.URI -import java.text.SimpleDateFormat -import java.util.Date - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} -import org.apache.hadoop.fs.permission.FsPermission - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.io.CompressionCodec - -/** - * A generic class for logging information to file. - * - * @param logDir Path to the directory in which files are logged - * @param outputBufferSize The buffer size to use when writing to an output stream in bytes - * @param compress Whether to compress output - * @param overwrite Whether to overwrite existing files - */ -private[spark] class FileLogger( - logDir: String, - sparkConf: SparkConf, - hadoopConf: Configuration, - outputBufferSize: Int = 8 * 1024, // 8 KB - compress: Boolean = false, - overwrite: Boolean = true, - dirPermissions: Option[FsPermission] = None) - extends Logging { - - def this( - logDir: String, - sparkConf: SparkConf, - compress: Boolean, - overwrite: Boolean) = { - this(logDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf), compress = compress, - overwrite = overwrite) - } - - def this( - logDir: String, - sparkConf: SparkConf, - compress: Boolean) = { - this(logDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf), compress = compress, - overwrite = true) - } - - def this( - logDir: String, - sparkConf: SparkConf) = { - this(logDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf), compress = false, - overwrite = true) - } - - private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - } - - /** - * To avoid effects of FileSystem#close or FileSystem.closeAll called from other modules, - * create unique FileSystem instance only for FileLogger - */ - private val fileSystem = { - val conf = SparkHadoopUtil.get.newConfiguration(sparkConf) - val logUri = new URI(logDir) - val scheme = logUri.getScheme - if (scheme == "hdfs") { - conf.setBoolean("fs.hdfs.impl.disable.cache", true) - } - FileSystem.get(logUri, conf) - } - - var fileIndex = 0 - - // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(sparkConf) - - // 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 - - /** - * Start this logger by creating the logging directory. - */ - def start() { - createLogDir() - } - - /** - * Create a logging directory with the given path. - */ - private def createLogDir() { - val path = new Path(logDir) - if (fileSystem.exists(path)) { - if (overwrite) { - logWarning("Log directory %s already exists. Overwriting...".format(logDir)) - // Second parameter is whether to delete recursively - fileSystem.delete(path, true) - } else { - throw new IOException("Log directory %s already exists!".format(logDir)) - } - } - if (!fileSystem.mkdirs(path)) { - throw new IOException("Error in creating log directory: %s".format(logDir)) - } - if (dirPermissions.isDefined) { - val fsStatus = fileSystem.getFileStatus(path) - if (fsStatus.getPermission.toShort != dirPermissions.get.toShort) { - fileSystem.setPermission(path, dirPermissions.get) - } - } - } - - /** - * Create a new writer for the file identified by the given path. - * If the permissions are not passed in, it will default to use the permissions - * (dirPermissions) used when class was instantiated. - */ - private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = { - val logPath = logDir + "/" + fileName - val uri = new URI(logPath) - val path = new Path(logPath) - val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme - val isDefaultLocal = defaultFs == null || defaultFs == "file" - - /* 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") { - // Second parameter is whether to append - new FileOutputStream(uri.getPath, !overwrite) - } else { - hadoopDataStream = Some(fileSystem.create(path, overwrite)) - hadoopDataStream.get - } - - perms.orElse(dirPermissions).foreach { p => fileSystem.setPermission(path, p) } - val bstream = new BufferedOutputStream(dstream, outputBufferSize) - val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream - new PrintWriter(cstream) - } - - /** - * Log the message to the given writer. - * @param msg The message to be logged - * @param withTime Whether to prepend message with a timestamp - */ - def log(msg: String, withTime: Boolean = false) { - val writeInfo = if (!withTime) { - msg - } else { - val date = new Date(System.currentTimeMillis) - dateFormat.get.format(date) + ": " + msg - } - writer.foreach(_.print(writeInfo)) - } - - /** - * Log the message to the given writer as a new line. - * @param msg The message to be logged - * @param withTime Whether to prepend message with a timestamp - */ - def logLine(msg: String, withTime: Boolean = false) = log(msg + "\n", withTime) - - /** - * Flush the writer to disk manually. - * - * When using a Hadoop filesystem, we need to invoke the hflush or sync - * method. In HDFS, hflush guarantees that the data gets to all the - * DataNodes. - */ - def flush() { - writer.foreach(_.flush()) - hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) - } - - /** - * Close the writer. Any subsequent calls to log or flush will have no effect. - */ - def close() { - writer.foreach(_.close()) - writer = None - } - - /** - * Start a writer for a new file, closing the existing one if it exists. - * @param fileName Name of the new file, defaulting to the file index if not provided. - * @param perms Permissions to put on the new file. - */ - def newFile(fileName: String = "", perms: Option[FsPermission] = None) { - fileIndex += 1 - writer.foreach(_.close()) - val name = fileName match { - case "" => fileIndex.toString - case _ => fileName - } - writer = Some(createWriter(name, perms)) - } - - /** - * Close all open writers, streams, and file systems. Any subsequent uses of this FileLogger - * instance will throw exceptions. - */ - def stop() { - hadoopDataStream.foreach(_.close()) - writer.foreach(_.close()) - } -} diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 7a7d4efb47aae..68a46bbcb7070 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -530,7 +530,7 @@ private[spark] object JsonProtocol { def stageInfoFromJson(json: JValue): StageInfo = { val stageId = (json \ "Stage ID").extract[Int] - val attemptId = (json \ "Attempt ID").extractOpt[Int].getOrElse(0) + val attemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson(_)) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 9c04e45a58479..c4f1898a2db15 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -246,8 +246,11 @@ private[spark] object Utils extends Logging { retval } - /** Create a temporary directory inside the given parent directory */ - def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + /** + * Create a directory inside the given parent directory. The directory is guaranteed to be + * newly created, and is not marked for automatic deletion. + */ + def createDirectory(root: String): File = { var attempts = 0 val maxAttempts = 10 var dir: File = null @@ -265,6 +268,15 @@ private[spark] object Utils extends Logging { } catch { case e: SecurityException => dir = null; } } + dir + } + + /** + * Create a temporary directory inside the given parent directory. The directory will be + * automatically deleted when the VM shuts down. + */ + def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + val dir = createDirectory(root) registerShutdownDeleteDir(dir) dir } @@ -385,16 +397,12 @@ private[spark] object Utils extends Logging { } finally { lock.release() } - if (targetFile.exists && !Files.equal(cachedFile, targetFile)) { - if (conf.getBoolean("spark.files.overwrite", false)) { - targetFile.delete() - logInfo((s"File $targetFile exists and does not match contents of $url, " + - s"replacing it with $url")) - } else { - throw new SparkException(s"File $targetFile exists and does not match contents of $url") - } - } - Files.copy(cachedFile, targetFile) + copyFile( + url, + cachedFile, + targetFile, + conf.getBoolean("spark.files.overwrite", false) + ) } else { doFetchFile(url, targetDir, fileName, conf, securityMgr, hadoopConf) } @@ -411,6 +419,104 @@ private[spark] object Utils extends Logging { FileUtil.chmod(targetFile.getAbsolutePath, "a+x") } + /** + * Download `in` to `tempFile`, then move it to `destFile`. + * + * If `destFile` already exists: + * - no-op if its contents equal those of `sourceFile`, + * - throw an exception if `fileOverwrite` is false, + * - attempt to overwrite it otherwise. + * + * @param url URL that `sourceFile` originated from, for logging purposes. + * @param in InputStream to download. + * @param tempFile File path to download `in` to. + * @param destFile File path to move `tempFile` to. + * @param fileOverwrite Whether to delete/overwrite an existing `destFile` that does not match + * `sourceFile` + */ + private def downloadFile( + url: String, + in: InputStream, + tempFile: File, + destFile: File, + fileOverwrite: Boolean): Unit = { + + try { + val out = new FileOutputStream(tempFile) + Utils.copyStream(in, out, closeStreams = true) + copyFile(url, tempFile, destFile, fileOverwrite, removeSourceFile = true) + } finally { + // Catch-all for the couple of cases where for some reason we didn't move `tempFile` to + // `destFile`. + if (tempFile.exists()) { + tempFile.delete() + } + } + } + + /** + * Copy `sourceFile` to `destFile`. + * + * If `destFile` already exists: + * - no-op if its contents equal those of `sourceFile`, + * - throw an exception if `fileOverwrite` is false, + * - attempt to overwrite it otherwise. + * + * @param url URL that `sourceFile` originated from, for logging purposes. + * @param sourceFile File path to copy/move from. + * @param destFile File path to copy/move to. + * @param fileOverwrite Whether to delete/overwrite an existing `destFile` that does not match + * `sourceFile` + * @param removeSourceFile Whether to remove `sourceFile` after / as part of moving/copying it to + * `destFile`. + */ + private def copyFile( + url: String, + sourceFile: File, + destFile: File, + fileOverwrite: Boolean, + removeSourceFile: Boolean = false): Unit = { + + if (destFile.exists) { + if (!Files.equal(sourceFile, destFile)) { + if (fileOverwrite) { + logInfo( + s"File $destFile exists and does not match contents of $url, replacing it with $url" + ) + if (!destFile.delete()) { + throw new SparkException( + "Failed to delete %s while attempting to overwrite it with %s".format( + destFile.getAbsolutePath, + sourceFile.getAbsolutePath + ) + ) + } + } else { + throw new SparkException( + s"File $destFile exists and does not match contents of $url") + } + } else { + // Do nothing if the file contents are the same, i.e. this file has been copied + // previously. + logInfo( + "%s has been previously copied to %s".format( + sourceFile.getAbsolutePath, + destFile.getAbsolutePath + ) + ) + return + } + } + + // The file does not exist in the target directory. Copy or move it there. + if (removeSourceFile) { + Files.move(sourceFile, destFile) + } else { + logInfo(s"Copying ${sourceFile.getAbsolutePath} to ${destFile.getAbsolutePath}") + Files.copy(sourceFile, destFile) + } + } + /** * Download a file to target directory. Supports fetching the file in a variety of ways, * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. @@ -449,67 +555,17 @@ private[spark] object Utils extends Logging { uc.setReadTimeout(timeout) uc.connect() val in = uc.getInputStream() - val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, closeStreams = true) - if (targetFile.exists && !Files.equal(tempFile, targetFile)) { - if (fileOverwrite) { - targetFile.delete() - logInfo(("File %s exists and does not match contents of %s, " + - "replacing it with %s").format(targetFile, url, url)) - } else { - tempFile.delete() - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } - } - Files.move(tempFile, targetFile) + downloadFile(url, in, tempFile, targetFile, fileOverwrite) case "file" => // In the case of a local file, copy the local file to the target directory. // Note the difference between uri vs url. val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) - var shouldCopy = true - if (targetFile.exists) { - if (!Files.equal(sourceFile, targetFile)) { - if (fileOverwrite) { - targetFile.delete() - logInfo(("File %s exists and does not match contents of %s, " + - "replacing it with %s").format(targetFile, url, url)) - } else { - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } - } else { - // Do nothing if the file contents are the same, i.e. this file has been copied - // previously. - logInfo(sourceFile.getAbsolutePath + " has been previously copied to " - + targetFile.getAbsolutePath) - shouldCopy = false - } - } - - if (shouldCopy) { - // The file does not exist in the target directory. Copy it there. - logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) - Files.copy(sourceFile, targetFile) - } + copyFile(url, sourceFile, targetFile, fileOverwrite) case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others val fs = getHadoopFileSystem(uri, hadoopConf) val in = fs.open(new Path(uri)) - val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, closeStreams = true) - if (targetFile.exists && !Files.equal(tempFile, targetFile)) { - if (fileOverwrite) { - targetFile.delete() - logInfo(("File %s exists and does not match contents of %s, " + - "replacing it with %s").format(targetFile, url, url)) - } else { - tempFile.delete() - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } - } - Files.move(tempFile, targetFile) + downloadFile(url, in, tempFile, targetFile, fileOverwrite) } } @@ -934,11 +990,12 @@ private[spark] object Utils extends Logging { for ((key, value) <- extraEnvironment) { environment.put(key, value) } + val process = builder.start() new Thread("read stderr for " + command(0)) { override def run() { for (line <- Source.fromInputStream(process.getErrorStream).getLines()) { - System.err.println(line) + logInfo(line) } } }.start() @@ -1024,13 +1081,6 @@ private[spark] object Utils extends Logging { * @param skipClass Function that is used to exclude non-user-code classes. */ def getCallSite(skipClass: String => Boolean = coreExclusionFunction): CallSite = { - val trace = Thread.currentThread.getStackTrace().filterNot { ste: StackTraceElement => - // When running under some profilers, the current stack trace might contain some bogus - // frames. This is intended to ensure that we don't crash in these situations by - // ignoring any frames that we can't examine. - ste == null || ste.getMethodName == null || ste.getMethodName.contains("getStackTrace") - } - // Keep crawling up the stack trace until we find the first function not inside of the spark // package. We track the last (shallowest) contiguous Spark method. This might be an RDD // transformation, a SparkContext function (such as parallelize), or anything else that leads @@ -1041,26 +1091,33 @@ private[spark] object Utils extends Logging { var insideSpark = true var callStack = new ArrayBuffer[String]() :+ "" - for (el <- trace) { - if (insideSpark) { - if (skipClass(el.getClassName)) { - lastSparkMethod = if (el.getMethodName == "") { - // Spark method is a constructor; get its class name - el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) + Thread.currentThread.getStackTrace().foreach { ste: StackTraceElement => + // When running under some profilers, the current stack trace might contain some bogus + // frames. This is intended to ensure that we don't crash in these situations by + // ignoring any frames that we can't examine. + if (ste != null && ste.getMethodName != null + && !ste.getMethodName.contains("getStackTrace")) { + if (insideSpark) { + if (skipClass(ste.getClassName)) { + lastSparkMethod = if (ste.getMethodName == "") { + // Spark method is a constructor; get its class name + ste.getClassName.substring(ste.getClassName.lastIndexOf('.') + 1) + } else { + ste.getMethodName + } + callStack(0) = ste.toString // Put last Spark method on top of the stack trace. } else { - el.getMethodName + firstUserLine = ste.getLineNumber + firstUserFile = ste.getFileName + callStack += ste.toString + insideSpark = false } - callStack(0) = el.toString // Put last Spark method on top of the stack trace. } else { - firstUserLine = el.getLineNumber - firstUserFile = el.getFileName - callStack += el.toString - insideSpark = false + callStack += ste.toString } - } else { - callStack += el.toString } } + val callStackDepth = System.getProperty("spark.callstack.depth", "20").toInt CallSite( shortForm = s"$lastSparkMethod at $firstUserFile:$firstUserLine", @@ -1786,24 +1843,63 @@ private[spark] object Utils extends Logging { sparkValue } } + + /** + * Return a pair of host and port extracted from the `sparkUrl`. + * + * A spark url (`spark://host:port`) is a special URI that its scheme is `spark` and only contains + * host and port. + * + * @throws SparkException if `sparkUrl` is invalid. + */ + def extractHostPortFromSparkUrl(sparkUrl: String): (String, Int) = { + try { + val uri = new java.net.URI(sparkUrl) + val host = uri.getHost + val port = uri.getPort + if (uri.getScheme != "spark" || + host == null || + port < 0 || + (uri.getPath != null && !uri.getPath.isEmpty) || // uri.getPath returns "" instead of null + uri.getFragment != null || + uri.getQuery != null || + uri.getUserInfo != null) { + throw new SparkException("Invalid master URL: " + sparkUrl) + } + (host, port) + } catch { + case e: java.net.URISyntaxException => + throw new SparkException("Invalid master URL: " + sparkUrl, e) + } + } } /** * A utility class to redirect the child process's stdout or stderr. */ -private[spark] class RedirectThread(in: InputStream, out: OutputStream, name: String) +private[spark] class RedirectThread( + in: InputStream, + out: OutputStream, + name: String, + propagateEof: Boolean = false) extends Thread(name) { setDaemon(true) override def run() { scala.util.control.Exception.ignoring(classOf[IOException]) { // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - out.write(buf, 0, len) - out.flush() - len = in.read(buf) + try { + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } finally { + if (propagateEof) { + out.close() + } } } } diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index c6cab82c3e546..2ed827eab46df 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -24,9 +24,9 @@ import org.apache.spark.util.random.XORShiftRandom @deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { - def length = elements.length + def length: Int = elements.length - def apply(index: Int) = elements(index) + def apply(index: Int): Double = elements(index) def + (other: Vector): Vector = { if (length != other.length) { @@ -35,7 +35,7 @@ class Vector(val elements: Array[Double]) extends Serializable { Vector(length, i => this(i) + other(i)) } - def add(other: Vector) = this + other + def add(other: Vector): Vector = this + other def - (other: Vector): Vector = { if (length != other.length) { @@ -44,7 +44,7 @@ class Vector(val elements: Array[Double]) extends Serializable { Vector(length, i => this(i) - other(i)) } - def subtract(other: Vector) = this - other + def subtract(other: Vector): Vector = this - other def dot(other: Vector): Double = { if (length != other.length) { @@ -93,19 +93,19 @@ class Vector(val elements: Array[Double]) extends Serializable { this } - def addInPlace(other: Vector) = this +=other + def addInPlace(other: Vector): Vector = this +=other def * (scale: Double): Vector = Vector(length, i => this(i) * scale) - def multiply (d: Double) = this * d + def multiply (d: Double): Vector = this * d def / (d: Double): Vector = this * (1 / d) - def divide (d: Double) = this / d + def divide (d: Double): Vector = this / d - def unary_- = this * -1 + def unary_- : Vector = this * -1 - def sum = elements.reduceLeft(_ + _) + def sum: Double = elements.reduceLeft(_ + _) def squaredDist(other: Vector): Double = { var ans = 0.0 @@ -119,40 +119,40 @@ class Vector(val elements: Array[Double]) extends Serializable { def dist(other: Vector): Double = math.sqrt(squaredDist(other)) - override def toString = elements.mkString("(", ", ", ")") + override def toString: String = elements.mkString("(", ", ", ")") } object Vector { - def apply(elements: Array[Double]) = new Vector(elements) + def apply(elements: Array[Double]): Vector = new Vector(elements) - def apply(elements: Double*) = new Vector(elements.toArray) + def apply(elements: Double*): Vector = new Vector(elements.toArray) def apply(length: Int, initializer: Int => Double): Vector = { val elements: Array[Double] = Array.tabulate(length)(initializer) new Vector(elements) } - def zeros(length: Int) = new Vector(new Array[Double](length)) + def zeros(length: Int): Vector = new Vector(new Array[Double](length)) - def ones(length: Int) = Vector(length, _ => 1) + def ones(length: Int): Vector = Vector(length, _ => 1) /** * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. */ - def random(length: Int, random: Random = new XORShiftRandom()) = + def random(length: Int, random: Random = new XORShiftRandom()): Vector = Vector(length, _ => random.nextDouble()) class Multiplier(num: Double) { - def * (vec: Vector) = vec * num + def * (vec: Vector): Vector = vec * num } - implicit def doubleToMultiplier(num: Double) = new Multiplier(num) + implicit def doubleToMultiplier(num: Double): Multiplier = new Multiplier(num) implicit object VectorAccumParam extends org.apache.spark.AccumulatorParam[Vector] { - def addInPlace(t1: Vector, t2: Vector) = t1 + t2 + def addInPlace(t1: Vector, t2: Vector): Vector = t1 + t2 - def zero(initialValue: Vector) = Vector.zeros(initialValue.length) + def zero(initialValue: Vector): Vector = Vector.zeros(initialValue.length) } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e5bdad6bda2fa..5ce299d05824b 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -184,6 +184,7 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } + @SuppressWarnings("unchecked") @Test public void repartitionAndSortWithinPartitions() { List> pairs = new ArrayList>(); @@ -491,6 +492,7 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(33, sum); } + @SuppressWarnings("unchecked") @Test public void aggregateByKey() { JavaPairRDD pairs = sc.parallelizePairs( @@ -1556,7 +1558,7 @@ static class Class2 {} @Test public void testRegisterKryoClasses() { SparkConf conf = new SparkConf(); - conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class }); + conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class }); Assert.assertEquals( Class1.class.getName() + "," + Class2.class.getName(), conf.get("spark.kryo.classesToRegister")); diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 9dd05f17f012b..287c8e3563503 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -15,10 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file core/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/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 998f3008ec0ea..97ea3578aa8ba 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark -import org.scalatest.BeforeAndAfter import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers @@ -29,16 +28,10 @@ class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} -class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter - with LocalSparkContext { +class DistributedSuite extends FunSuite with Matchers with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" - after { - System.clearProperty("spark.reducer.maxMbInFlight") - System.clearProperty("spark.storage.memoryFraction") - } - test("task throws not serializable exception") { // Ensures that executors do not crash when an exn is not serializable. If executors crash, // this test will hang. Correct behavior is that executors don't crash but fail tasks @@ -84,15 +77,14 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter } test("groupByKey where map output sizes exceed maxMbInFlight") { - System.setProperty("spark.reducer.maxMbInFlight", "1") - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().set("spark.reducer.maxMbInFlight", "1") + sc = new SparkContext(clusterUrl, "test", conf) // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output // file should be about 2.5 MB val pairs = sc.parallelize(1 to 2000, 4).map(x => (x % 16, new Array[Byte](10000))) val groups = pairs.groupByKey(2).map(x => (x._1, x._2.size)).collect() assert(groups.length === 16) assert(groups.map(_._2).sum === 2000) - // Note that spark.reducer.maxMbInFlight will be cleared in the test suite's after{} block } test("accumulators") { @@ -210,7 +202,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter } test("compute without caching when no partitions fit in memory") { - System.setProperty("spark.storage.memoryFraction", "0.0001") sc = new SparkContext(clusterUrl, "test") // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache // to only 50 KB (0.0001 of 512 MB), so no partitions should fit in memory @@ -218,12 +209,11 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter assert(data.count() === 4000000) assert(data.count() === 4000000) assert(data.count() === 4000000) - System.clearProperty("spark.storage.memoryFraction") } test("compute when only some partitions fit in memory") { - System.setProperty("spark.storage.memoryFraction", "0.01") - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().set("spark.storage.memoryFraction", "0.01") + sc = new SparkContext(clusterUrl, "test", conf) // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache // to only 5 MB (0.01 of 512 MB), so not all of it will fit in memory; we use 20 partitions // to make sure that *some* of them do fit though @@ -231,7 +221,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter assert(data.count() === 4000000) assert(data.count() === 4000000) assert(data.count() === 4000000) - System.clearProperty("spark.storage.memoryFraction") } test("passing environment variables to cluster") { diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 5265ba904032f..8a54360e81795 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -35,7 +35,7 @@ class DriverSuite extends FunSuite with Timeouts { forAll(masters) { (master: String) => failAfter(60 seconds) { Utils.executeAndGetOutput( - Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), + Seq(s"$sparkHome/bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } @@ -50,7 +50,10 @@ class DriverSuite extends FunSuite with Timeouts { object DriverWithoutCleanup { def main(args: Array[String]) { Utils.configTestLog4j("INFO") - val sc = new SparkContext(args(0), "DriverWithoutCleanup") + // Bind the web UI to an ephemeral port in order to avoid conflicts with other tests running on + // the same machine (we shouldn't just disable the UI here, since that might mask bugs): + val conf = new SparkConf().set("spark.ui.port", "0") + val sc = new SparkContext(args(0), "DriverWithoutCleanup", conf) sc.parallelize(1 to 100, 4).count() } } diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 49426545c767e..0f49ce4754fbb 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -31,10 +31,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ @transient var tmpJarUrl: String = _ + def newConf: SparkConf = new SparkConf(loadDefaults = false).set("spark.authenticate", "false") + override def beforeEach() { super.beforeEach() resetSparkContext() - System.setProperty("spark.authenticate", "false") } override def beforeAll() { @@ -52,7 +53,6 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val jarFile = new File(testTempDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) - System.setProperty("spark.authenticate", "false") val jarEntry = new JarEntry(textFile.getName) jar.putNextEntry(jarEntry) @@ -74,7 +74,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test("Distributing files locally") { - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -108,7 +108,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test("Distributing files locally using URL as input") { // addFile("file:///....") - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addFile(new File(tmpFile.toString).toURI.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -122,7 +122,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS locally") { - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addJar(tmpJarUrl) val testData = Array((1, 1)) sc.parallelize(testData).foreach { x => @@ -133,7 +133,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test("Distributing files on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -147,7 +147,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addJar(tmpJarUrl) val testData = Array((1,1)) sc.parallelize(testData).foreach { x => @@ -158,7 +158,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS on a standalone cluster using local: URL") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addJar(tmpJarUrl.replace("file", "local")) val testData = Array((1,1)) sc.parallelize(testData).foreach { x => diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 41ed2bce55ce1..7584ae79fc920 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -40,12 +40,11 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter override def afterEach() { super.afterEach() resetSparkContext() - System.clearProperty("spark.scheduler.mode") } test("local mode, FIFO scheduler") { - System.setProperty("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local[2]", "test") + val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + sc = new SparkContext("local[2]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -53,10 +52,10 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("local mode, fair scheduler") { - System.setProperty("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local[2]", "test") + conf.set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local[2]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -64,8 +63,8 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("cluster mode, FIFO scheduler") { - System.setProperty("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local-cluster[2,1,512]", "test") + val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -73,10 +72,10 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("cluster mode, fair scheduler") { - System.setProperty("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local-cluster[2,1,512]", "test") + conf.set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 58a96245a9b53..f57921b768310 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -35,19 +35,15 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex conf.set("spark.test.noStageRetry", "true") test("groupByKey without compression") { - try { - System.setProperty("spark.shuffle.compress", "false") - sc = new SparkContext("local", "test", conf) - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) - val groups = pairs.groupByKey(4).collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } finally { - System.setProperty("spark.shuffle.compress", "true") - } + val myConf = conf.clone().set("spark.shuffle.compress", "false") + sc = new SparkContext("local", "test", myConf) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) + val groups = pairs.groupByKey(4).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) } test("shuffle non-zero block size") { diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 5d018ea9868a7..790976a5ac308 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -19,27 +19,20 @@ package org.apache.spark import org.scalatest.FunSuite import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} +import org.apache.spark.util.ResetSystemProperties import com.esotericsoftware.kryo.Kryo -class SparkConfSuite extends FunSuite with LocalSparkContext { +class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { test("loading from system properties") { - try { - System.setProperty("spark.test.testProperty", "2") - val conf = new SparkConf() - assert(conf.get("spark.test.testProperty") === "2") - } finally { - System.clearProperty("spark.test.testProperty") - } + System.setProperty("spark.test.testProperty", "2") + val conf = new SparkConf() + assert(conf.get("spark.test.testProperty") === "2") } test("initializing without loading defaults") { - try { - System.setProperty("spark.test.testProperty", "2") - val conf = new SparkConf(false) - assert(!conf.contains("spark.test.testProperty")) - } finally { - System.clearProperty("spark.test.testProperty") - } + System.setProperty("spark.test.testProperty", "2") + val conf = new SparkConf(false) + assert(!conf.contains("spark.test.testProperty")) } test("named set methods") { @@ -117,23 +110,17 @@ class SparkConfSuite extends FunSuite with LocalSparkContext { test("nested property names") { // This wasn't supported by some external conf parsing libraries - try { - System.setProperty("spark.test.a", "a") - System.setProperty("spark.test.a.b", "a.b") - System.setProperty("spark.test.a.b.c", "a.b.c") - val conf = new SparkConf() - assert(conf.get("spark.test.a") === "a") - assert(conf.get("spark.test.a.b") === "a.b") - assert(conf.get("spark.test.a.b.c") === "a.b.c") - conf.set("spark.test.a.b", "A.B") - assert(conf.get("spark.test.a") === "a") - assert(conf.get("spark.test.a.b") === "A.B") - assert(conf.get("spark.test.a.b.c") === "a.b.c") - } finally { - System.clearProperty("spark.test.a") - System.clearProperty("spark.test.a.b") - System.clearProperty("spark.test.a.b.c") - } + System.setProperty("spark.test.a", "a") + System.setProperty("spark.test.a.b", "a.b") + System.setProperty("spark.test.a.b.c", "a.b.c") + val conf = new SparkConf() + assert(conf.get("spark.test.a") === "a") + assert(conf.get("spark.test.a.b") === "a.b") + assert(conf.get("spark.test.a.b.c") === "a.b.c") + conf.set("spark.test.a.b", "A.B") + assert(conf.get("spark.test.a") === "a") + assert(conf.get("spark.test.a.b") === "A.B") + assert(conf.get("spark.test.a.b.c") === "a.b.c") } test("register kryo classes through registerKryoClasses") { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 0390a2e4f1dbb..8ae4f243ec1ae 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -27,10 +27,13 @@ import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite extends FunSuite with LocalSparkContext with PrivateMethodTester with Logging { - def createTaskScheduler(master: String): TaskSchedulerImpl = { + def createTaskScheduler(master: String): TaskSchedulerImpl = + createTaskScheduler(master, new SparkConf()) + + def createTaskScheduler(master: String, conf: SparkConf): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val createTaskSchedulerMethod = PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) @@ -102,19 +105,13 @@ class SparkContextSchedulerCreationSuite } test("local-default-parallelism") { - val defaultParallelism = System.getProperty("spark.default.parallelism") - System.setProperty("spark.default.parallelism", "16") - val sched = createTaskScheduler("local") + val conf = new SparkConf().set("spark.default.parallelism", "16") + val sched = createTaskScheduler("local", conf) sched.backend match { case s: LocalBackend => assert(s.defaultParallelism() === 16) case _ => fail() } - - Option(defaultParallelism) match { - case Some(v) => System.setProperty("spark.default.parallelism", v) - case _ => System.clearProperty("spark.default.parallelism") - } } test("simr") { @@ -155,9 +152,10 @@ class SparkContextSchedulerCreationSuite testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") } - def testMesos(master: String, expectedClass: Class[_]) { + def testMesos(master: String, expectedClass: Class[_], coarse: Boolean) { + val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString) try { - val sched = createTaskScheduler(master) + val sched = createTaskScheduler(master, conf) assert(sched.backend.getClass === expectedClass) } catch { case e: UnsatisfiedLinkError => @@ -168,17 +166,14 @@ class SparkContextSchedulerCreationSuite } test("mesos fine-grained") { - System.setProperty("spark.mesos.coarse", "false") - testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend]) + testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false) } test("mesos coarse-grained") { - System.setProperty("spark.mesos.coarse", "true") - testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend]) + testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true) } test("mesos with zookeeper") { - System.setProperty("spark.mesos.coarse", "false") - testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend]) + testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false) } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 1362022104195..8b3c6871a7b39 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -23,55 +23,37 @@ import org.apache.hadoop.io.BytesWritable class SparkContextSuite extends FunSuite with LocalSparkContext { - /** Allows system properties to be changed in tests */ - private def withSystemProperty[T](property: String, value: String)(block: => T): T = { - val originalValue = System.getProperty(property) - try { - System.setProperty(property, value) - block - } finally { - if (originalValue == null) { - System.clearProperty(property) - } else { - System.setProperty(property, originalValue) - } - } - } - test("Only one SparkContext may be active at a time") { // Regression test for SPARK-4180 - withSystemProperty("spark.driver.allowMultipleContexts", "false") { - val conf = new SparkConf().setAppName("test").setMaster("local") - sc = new SparkContext(conf) - // A SparkContext is already running, so we shouldn't be able to create a second one - intercept[SparkException] { new SparkContext(conf) } - // After stopping the running context, we should be able to create a new one - resetSparkContext() - sc = new SparkContext(conf) - } + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "false") + sc = new SparkContext(conf) + // A SparkContext is already running, so we shouldn't be able to create a second one + intercept[SparkException] { new SparkContext(conf) } + // After stopping the running context, we should be able to create a new one + resetSparkContext() + sc = new SparkContext(conf) } test("Can still construct a new SparkContext after failing to construct a previous one") { - withSystemProperty("spark.driver.allowMultipleContexts", "false") { - // This is an invalid configuration (no app name or master URL) - intercept[SparkException] { - new SparkContext(new SparkConf()) - } - // Even though those earlier calls failed, we should still be able to create a new context - sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test")) + val conf = new SparkConf().set("spark.driver.allowMultipleContexts", "false") + // This is an invalid configuration (no app name or master URL) + intercept[SparkException] { + new SparkContext(conf) } + // Even though those earlier calls failed, we should still be able to create a new context + sc = new SparkContext(conf.setMaster("local").setAppName("test")) } test("Check for multiple SparkContexts can be disabled via undocumented debug option") { - withSystemProperty("spark.driver.allowMultipleContexts", "true") { - var secondSparkContext: SparkContext = null - try { - val conf = new SparkConf().setAppName("test").setMaster("local") - sc = new SparkContext(conf) - secondSparkContext = new SparkContext(conf) - } finally { - Option(secondSparkContext).foreach(_.stop()) - } + var secondSparkContext: SparkContext = null + try { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true") + sc = new SparkContext(conf) + secondSparkContext = new SparkContext(conf) + } finally { + Option(secondSparkContext).foreach(_.stop()) } } diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala new file mode 100644 index 0000000000000..8959a843dbd7d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala @@ -0,0 +1,60 @@ +/* + * 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.api.python + +import scala.io.Source + +import java.io.{PrintWriter, File} + +import org.scalatest.{Matchers, FunSuite} + +import org.apache.spark.{SharedSparkContext, SparkConf} +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.util.Utils + +// This test suite uses SharedSparkContext because we need a SparkEnv in order to deserialize +// a PythonBroadcast: +class PythonBroadcastSuite extends FunSuite with Matchers with SharedSparkContext { + test("PythonBroadcast can be serialized with Kryo (SPARK-4882)") { + val tempDir = Utils.createTempDir() + val broadcastedString = "Hello, world!" + def assertBroadcastIsValid(broadcast: PythonBroadcast): Unit = { + val source = Source.fromFile(broadcast.path) + val contents = source.mkString + source.close() + contents should be (broadcastedString) + } + try { + val broadcastDataFile: File = { + val file = new File(tempDir, "broadcastData") + val printWriter = new PrintWriter(file) + printWriter.write(broadcastedString) + printWriter.close() + file + } + val broadcast = new PythonBroadcast(broadcastDataFile.getAbsolutePath) + assertBroadcastIsValid(broadcast) + val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val deserializedBroadcast = + Utils.clone[PythonBroadcast](broadcast, new KryoSerializer(conf).newInstance()) + assertBroadcastIsValid(deserializedBroadcast) + } finally { + Utils.deleteRecursively(tempDir) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 3f1cd0752e766..aa65f7e8915e6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -119,7 +119,7 @@ class JsonProtocolSuite extends FunSuite { def createExecutorRunner(): ExecutorRunner = { new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", new File("sparkHome"), new File("workDir"), "akka://worker", - new SparkConf, ExecutorState.RUNNING) + new SparkConf, Seq("localDir"), ExecutorState.RUNNING) } def createDriverRunner(): DriverRunner = { diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index eb7bd7ab3986e..065b7534cece6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -23,11 +23,13 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.deploy.SparkSubmit._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ResetSystemProperties, Utils} import org.scalatest.FunSuite import org.scalatest.Matchers -class SparkSubmitSuite extends FunSuite with Matchers { +// Note: this suite mixes in ResetSystemProperties because SparkSubmit.main() sets a bunch +// of properties that neeed to be cleared after tests. +class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties { def beforeAll() { System.setProperty("spark.testing", "true") } @@ -288,6 +290,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", "--master", "local", + "--conf", "spark.ui.enabled=false", unusedJar.toString) runSparkSubmit(args) } @@ -302,6 +305,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--name", "testApp", "--master", "local-cluster[2,1,512]", "--jars", jarsString, + "--conf", "spark.ui.enabled=false", unusedJar.toString) runSparkSubmit(args) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala new file mode 100644 index 0000000000000..8379883e065e7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import java.io.{File, FileOutputStream, OutputStreamWriter} + +import scala.io.Source + +import com.google.common.io.Files +import org.apache.hadoop.fs.Path +import org.json4s.jackson.JsonMethods._ +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.Matchers + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.io._ +import org.apache.spark.scheduler._ +import org.apache.spark.util.{JsonProtocol, Utils} + +class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { + + private var testDir: File = null + + private var provider: FsHistoryProvider = null + + before { + testDir = Utils.createTempDir() + provider = new FsHistoryProvider(new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set("spark.history.fs.updateInterval", "0")) + } + + after { + Utils.deleteRecursively(testDir) + } + + test("Parse new and old application logs") { + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set("spark.history.fs.updateInterval", "0") + val provider = new FsHistoryProvider(conf) + + // Write a new-style application log. + val logFile1 = new File(testDir, "new1") + writeFile(logFile1, true, None, + SparkListenerApplicationStart("app1-1", None, 1L, "test"), + SparkListenerApplicationEnd(2L) + ) + + // Write an unfinished app, new-style. + val logFile2 = new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS) + writeFile(logFile2, true, None, + SparkListenerApplicationStart("app2-2", None, 1L, "test") + ) + + // Write an old-style application log. + val oldLog = new File(testDir, "old1") + oldLog.mkdir() + createEmptyFile(new File(oldLog, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldLog, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("app3", None, 2L, "test"), + SparkListenerApplicationEnd(3L) + ) + createEmptyFile(new File(oldLog, provider.APPLICATION_COMPLETE)) + + // Write an unfinished app, old-style. + val oldLog2 = new File(testDir, "old2") + oldLog2.mkdir() + createEmptyFile(new File(oldLog2, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldLog2, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("app4", None, 2L, "test") + ) + + // Force a reload of data from the log directory, and check that both logs are loaded. + // Take the opportunity to check that the offset checks work as expected. + provider.checkForLogs() + + val list = provider.getListing().toSeq + list should not be (null) + list.size should be (4) + list.count(e => e.completed) should be (2) + + list(0) should be (ApplicationHistoryInfo(oldLog.getName(), "app3", 2L, 3L, + oldLog.lastModified(), "test", true)) + list(1) should be (ApplicationHistoryInfo(logFile1.getName(), "app1-1", 1L, 2L, + logFile1.lastModified(), "test", true)) + list(2) should be (ApplicationHistoryInfo(oldLog2.getName(), "app4", 2L, -1L, + oldLog2.lastModified(), "test", false)) + list(3) should be (ApplicationHistoryInfo(logFile2.getName(), "app2-2", 1L, -1L, + logFile2.lastModified(), "test", false)) + + // Make sure the UI can be rendered. + list.foreach { case info => + val appUi = provider.getAppUI(info.id) + appUi should not be null + } + } + + test("Parse legacy logs with compression codec set") { + val testCodecs = List((classOf[LZFCompressionCodec].getName(), true), + (classOf[SnappyCompressionCodec].getName(), true), + ("invalid.codec", false)) + + testCodecs.foreach { case (codecName, valid) => + val codec = if (valid) CompressionCodec.createCodec(new SparkConf(), codecName) else null + val logDir = new File(testDir, codecName) + logDir.mkdir() + createEmptyFile(new File(logDir, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(logDir, provider.LOG_PREFIX + "1"), false, Option(codec), + SparkListenerApplicationStart("app2", None, 2L, "test"), + SparkListenerApplicationEnd(3L) + ) + createEmptyFile(new File(logDir, provider.COMPRESSION_CODEC_PREFIX + codecName)) + + val logPath = new Path(logDir.getAbsolutePath()) + try { + val (logInput, sparkVersion) = provider.openLegacyEventLog(logPath) + try { + Source.fromInputStream(logInput).getLines().toSeq.size should be (2) + } finally { + logInput.close() + } + } catch { + case e: IllegalArgumentException => + valid should be (false) + } + } + } + + test("SPARK-3697: ignore directories that cannot be read.") { + val logFile1 = new File(testDir, "new1") + writeFile(logFile1, true, None, + SparkListenerApplicationStart("app1-1", None, 1L, "test"), + SparkListenerApplicationEnd(2L) + ) + val logFile2 = new File(testDir, "new2") + writeFile(logFile2, true, None, + SparkListenerApplicationStart("app1-2", None, 1L, "test"), + SparkListenerApplicationEnd(2L) + ) + logFile2.setReadable(false, false) + + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set("spark.history.fs.updateInterval", "0") + val provider = new FsHistoryProvider(conf) + provider.checkForLogs() + + val list = provider.getListing().toSeq + list should not be (null) + list.size should be (1) + } + + private def writeFile(file: File, isNewFormat: Boolean, codec: Option[CompressionCodec], + events: SparkListenerEvent*) = { + val out = + if (isNewFormat) { + EventLoggingListener.initEventLog(new FileOutputStream(file), codec) + } else { + val fileStream = new FileOutputStream(file) + codec.map(_.compressedOutputStream(fileStream)).getOrElse(fileStream) + } + val writer = new OutputStreamWriter(out, "UTF-8") + try { + events.foreach(e => writer.write(compact(render(JsonProtocol.sparkEventToJson(e))) + "\n")) + } finally { + writer.close() + } + } + + private def createEmptyFile(file: File) = { + new FileOutputStream(file).close() + } + +} diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala new file mode 100644 index 0000000000000..3d2335f9b3637 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.master + +import akka.actor.Address +import org.scalatest.FunSuite + +import org.apache.spark.SparkException + +class MasterSuite extends FunSuite { + + test("toAkkaUrl") { + val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234") + assert("akka.tcp://sparkMaster@1.2.3.4:1234/user/Master" === akkaUrl) + } + + test("toAkkaUrl: a typo url") { + val e = intercept[SparkException] { + Master.toAkkaUrl("spark://1.2. 3.4:1234") + } + assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage) + } + + test("toAkkaAddress") { + val address = Master.toAkkaAddress("spark://1.2.3.4:1234") + assert(Address("akka.tcp", "sparkMaster", "1.2.3.4", 1234) === address) + } + + test("toAkkaAddress: a typo url") { + val e = intercept[SparkException] { + Master.toAkkaAddress("spark://1.2. 3.4:1234") + } + assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 196217062991e..6f233d7cf97aa 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -33,7 +33,8 @@ class ExecutorRunnerTest extends FunSuite { val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", - new File(sparkHome), new File("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) + new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), + ExecutorState.RUNNING) val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) assert(builder.command().last === appId) } diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 25be7f25c21bb..8c6035fb367fe 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -85,4 +85,10 @@ class CompressionCodecSuite extends FunSuite { assert(codec.getClass === classOf[SnappyCompressionCodec]) testCodec(codec) } + + test("bad compression codec") { + intercept[IllegalArgumentException] { + CompressionCodec.createCodec(conf, "foobar") + } + } } diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index ca226fd4e694f..f8bcde12a371a 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -24,14 +24,14 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.{SparkListenerTaskEnd, SparkListener} import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} import scala.collection.mutable.ArrayBuffer -class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with ShouldMatchers { +class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with Matchers { test("input metrics when reading text file with single split") { val file = new File(getClass.getSimpleName + ".txt") val pw = new PrintWriter(new FileWriter(file)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 436eea4f1fdcf..d30eb10bbe947 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -247,6 +247,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F test("[SPARK-3353] parent stage should have lower stage id") { sparkListener.stageByOrderOfExecution.clear() sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _, 4).count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.stageByOrderOfExecution.length === 2) assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) } @@ -739,7 +740,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F test("accumulator not calculated for resubmitted result stage") { //just for register - val accum = new Accumulator[Int](0, SparkContext.IntAccumulatorParam) + val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) val finalRdd = new MyRDD(sc, 1, Nil) submit(finalRdd, Array(0)) completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index abc300fcffaf9..1de7e130039a5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -17,69 +17,59 @@ package org.apache.spark.scheduler +import java.io.{File, FileOutputStream, InputStream, IOException} + import scala.collection.mutable import scala.io.Source -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.io.CompressionCodec -import org.apache.spark.SPARK_VERSION +import org.apache.spark.io._ import org.apache.spark.util.{JsonProtocol, Utils} -import java.io.File - /** * Test whether EventLoggingListener logs events properly. * - * This tests whether EventLoggingListener actually creates special files while logging events, - * whether the parsing of these special files is correct, and whether the logged events can be - * read and deserialized into actual SparkListenerEvents. + * This tests whether EventLoggingListener actually log files with expected name patterns while + * logging events, whether the parsing of the file names is correct, and whether the logged events + * can be read and deserialized into actual SparkListenerEvents. */ -class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { +class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Logging { + import EventLoggingListenerSuite._ + private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) - private val allCompressionCodecs = Seq[String]( - "org.apache.spark.io.LZFCompressionCodec", - "org.apache.spark.io.SnappyCompressionCodec" - ) private var testDir: File = _ - private var logDirPath: Path = _ + private var testDirPath: Path = _ before { testDir = Utils.createTempDir() - logDirPath = Utils.getFilePath(testDir, "spark-events") + testDir.deleteOnExit() + testDirPath = new Path(testDir.getAbsolutePath()) } after { Utils.deleteRecursively(testDir) } - test("Parse names of special files") { - testParsingFileName() - } - - test("Verify special files exist") { - testSpecialFilesExist() - } - - test("Verify special files exist with compression") { - allCompressionCodecs.foreach { codec => - testSpecialFilesExist(compressionCodec = Some(codec)) - } - } + test("Verify log file exist") { + // Verify logging directory exists + val conf = getLoggingConf(testDirPath) + val eventLogger = new EventLoggingListener("test", testDirPath.toUri().toString(), conf) + eventLogger.start() - test("Parse event logging info") { - testParsingLogInfo() - } + val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) + assert(fileSystem.exists(logPath)) + val logStatus = fileSystem.getFileStatus(logPath) + assert(!logStatus.isDir) - test("Parse event logging info with compression") { - allCompressionCodecs.foreach { codec => - testParsingLogInfo(compressionCodec = Some(codec)) - } + // Verify log is renamed after stop() + eventLogger.stop() + assert(!fileSystem.getFileStatus(new Path(eventLogger.logPath)).isDir) } test("Basic event logging") { @@ -87,7 +77,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { } test("Basic event logging with compression") { - allCompressionCodecs.foreach { codec => + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testEventLogging(compressionCodec = Some(codec)) } } @@ -97,11 +87,25 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { } test("End-to-end event logging with compression") { - allCompressionCodecs.foreach { codec => + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testApplicationEventLogging(compressionCodec = Some(codec)) } } + test("Log overwriting") { + val log = new FileOutputStream(new File(testDir, "test")) + log.close() + try { + testEventLogging() + assert(false) + } catch { + case e: IOException => + // Expected, since we haven't enabled log overwrite. + } + + // Try again, but enable overwriting. + testEventLogging(extraConf = Map("spark.eventLog.overwrite" -> "true")) + } /* ----------------- * * Actual test logic * @@ -109,130 +113,19 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { import EventLoggingListenerSuite._ - /** - * Test whether names of special files are correctly identified and parsed. - */ - private def testParsingFileName() { - val logPrefix = EventLoggingListener.LOG_PREFIX - val sparkVersionPrefix = EventLoggingListener.SPARK_VERSION_PREFIX - val compressionCodecPrefix = EventLoggingListener.COMPRESSION_CODEC_PREFIX - val applicationComplete = EventLoggingListener.APPLICATION_COMPLETE - assert(EventLoggingListener.isEventLogFile(logPrefix + "0")) - assert(EventLoggingListener.isEventLogFile(logPrefix + "100")) - assert(EventLoggingListener.isEventLogFile(logPrefix + "ANYTHING")) - assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "0.9.1")) - assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "1.0.0")) - assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "ANYTHING")) - assert(EventLoggingListener.isApplicationCompleteFile(applicationComplete)) - allCompressionCodecs.foreach { codec => - assert(EventLoggingListener.isCompressionCodecFile(compressionCodecPrefix + codec)) - } - - // Negatives - assert(!EventLoggingListener.isEventLogFile("The greatest man of all mankind")) - assert(!EventLoggingListener.isSparkVersionFile("Will never falter in the face of death!")) - assert(!EventLoggingListener.isCompressionCodecFile("Unless he chooses to leave behind")) - assert(!EventLoggingListener.isApplicationCompleteFile("The very treasure he calls Macbeth")) - - // Verify that parsing is correct - assert(EventLoggingListener.parseSparkVersion(sparkVersionPrefix + "1.0.0") === "1.0.0") - allCompressionCodecs.foreach { codec => - assert(EventLoggingListener.parseCompressionCodec(compressionCodecPrefix + codec) === codec) - } - } - - /** - * Test whether the special files produced by EventLoggingListener exist. - * - * There should be exactly one event log and one spark version file throughout the entire - * execution. If a compression codec is specified, then the compression codec file should - * also exist. Only after the application has completed does the test expect the application - * completed file to be present. - */ - private def testSpecialFilesExist(compressionCodec: Option[String] = None) { - - def assertFilesExist(logFiles: Array[FileStatus], loggerStopped: Boolean) { - val numCompressionCodecFiles = if (compressionCodec.isDefined) 1 else 0 - val numApplicationCompleteFiles = if (loggerStopped) 1 else 0 - assert(logFiles.size === 2 + numCompressionCodecFiles + numApplicationCompleteFiles) - assert(eventLogsExist(logFiles)) - assert(sparkVersionExists(logFiles)) - assert(compressionCodecExists(logFiles) === compressionCodec.isDefined) - assert(applicationCompleteExists(logFiles) === loggerStopped) - assertSparkVersionIsValid(logFiles) - compressionCodec.foreach { codec => - assertCompressionCodecIsValid(logFiles, codec) - } - } - - // Verify logging directory exists - val conf = getLoggingConf(logDirPath, compressionCodec) - val logBaseDir = conf.get("spark.eventLog.dir") - val appId = EventLoggingListenerSuite.getUniqueApplicationId - val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) - eventLogger.start() - val logPath = new Path(eventLogger.logDir) - assert(fileSystem.exists(logPath)) - val logDir = fileSystem.getFileStatus(logPath) - assert(logDir.isDir) - - // Verify special files are as expected before stop() - var logFiles = fileSystem.listStatus(logPath) - assert(logFiles != null) - assertFilesExist(logFiles, loggerStopped = false) - - // Verify special files are as expected after stop() - eventLogger.stop() - logFiles = fileSystem.listStatus(logPath) - assertFilesExist(logFiles, loggerStopped = true) - } - - /** - * Test whether EventLoggingListener correctly parses the correct information from the logs. - * - * This includes whether it returns the correct Spark version, compression codec (if any), - * and the application's completion status. - */ - private def testParsingLogInfo(compressionCodec: Option[String] = None) { - - def assertInfoCorrect(info: EventLoggingInfo, loggerStopped: Boolean) { - assert(info.logPaths.size > 0) - assert(info.sparkVersion === SPARK_VERSION) - assert(info.compressionCodec.isDefined === compressionCodec.isDefined) - info.compressionCodec.foreach { codec => - assert(compressionCodec.isDefined) - val expectedCodec = compressionCodec.get.split('.').last - assert(codec.getClass.getSimpleName === expectedCodec) - } - assert(info.applicationComplete === loggerStopped) - } - - // Verify that all information is correctly parsed before stop() - val conf = getLoggingConf(logDirPath, compressionCodec) - val logBaseDir = conf.get("spark.eventLog.dir") - val appId = EventLoggingListenerSuite.getUniqueApplicationId - val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) - eventLogger.start() - var eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assertInfoCorrect(eventLoggingInfo, loggerStopped = false) - - // Verify that all information is correctly parsed after stop() - eventLogger.stop() - eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assertInfoCorrect(eventLoggingInfo, loggerStopped = true) - } - /** * Test basic event logging functionality. * * This creates two simple events, posts them to the EventLoggingListener, and verifies that * exactly these two events are logged in the expected file. */ - private def testEventLogging(compressionCodec: Option[String] = None) { - val conf = getLoggingConf(logDirPath, compressionCodec) - val logBaseDir = conf.get("spark.eventLog.dir") - val appId = EventLoggingListenerSuite.getUniqueApplicationId - val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) + private def testEventLogging( + compressionCodec: Option[String] = None, + extraConf: Map[String, String] = Map()) { + val conf = getLoggingConf(testDirPath, compressionCodec) + extraConf.foreach { case (k, v) => conf.set(k, v) } + val logName = compressionCodec.map("test-" + _).getOrElse("test") + val eventLogger = new EventLoggingListener(logName, testDirPath.toUri().toString(), conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey") @@ -244,17 +137,21 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { listenerBus.addListener(eventLogger) listenerBus.postToAll(applicationStart) listenerBus.postToAll(applicationEnd) + eventLogger.stop() // Verify file contains exactly the two events logged - val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assert(eventLoggingInfo.logPaths.size > 0) - val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) - assert(lines.size === 2) - assert(lines(0).contains("SparkListenerApplicationStart")) - assert(lines(1).contains("SparkListenerApplicationEnd")) - assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) - assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) - eventLogger.stop() + val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), + fileSystem) + try { + val lines = readLines(logData) + assert(lines.size === 2) + assert(lines(0).contains("SparkListenerApplicationStart")) + assert(lines(1).contains("SparkListenerApplicationEnd")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) + } finally { + logData.close() + } } /** @@ -262,12 +159,12 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { * This runs a simple Spark job and asserts that the expected events are logged when expected. */ private def testApplicationEventLogging(compressionCodec: Option[String] = None) { - val conf = getLoggingConf(logDirPath, compressionCodec) + val conf = getLoggingConf(testDirPath, compressionCodec) val sc = new SparkContext("local", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get - val expectedLogDir = logDirPath.toString - assert(eventLogger.logDir.contains(expectedLogDir)) + val expectedLogDir = testDir.toURI().toString() + assert(eventLogger.logPath.startsWith(expectedLogDir + "/")) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) @@ -279,16 +176,22 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { // Ensure all asserts have actually been triggered eventExistenceListener.assertAllCallbacksInvoked() - } - /** - * Assert that all of the specified events are logged by the given EventLoggingListener. - */ - private def assertEventsExist(eventLogger: EventLoggingListener, events: Seq[String]) { - val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assert(eventLoggingInfo.logPaths.size > 0) - val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) - val eventSet = mutable.Set(events: _*) + // Make sure expected events exist in the log file. + val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), + fileSystem) + val lines = readLines(logData) + val eventSet = mutable.Set( + SparkListenerApplicationStart, + SparkListenerBlockManagerAdded, + SparkListenerEnvironmentUpdate, + SparkListenerJobStart, + SparkListenerJobEnd, + SparkListenerStageSubmitted, + SparkListenerStageCompleted, + SparkListenerTaskStart, + SparkListenerTaskEnd, + SparkListenerApplicationEnd).map(Utils.getFormattedClassName) lines.foreach { line => eventSet.foreach { event => if (line.contains(event)) { @@ -303,19 +206,8 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) } - /** - * Read all lines from the file specified by the given path. - * If a compression codec is specified, use it to read the file. - */ - private def readFileLines( - filePath: Path, - compressionCodec: Option[CompressionCodec]): Seq[String] = { - val fstream = fileSystem.open(filePath) - val cstream = - compressionCodec.map { codec => - codec.compressedInputStream(fstream) - }.getOrElse(fstream) - Source.fromInputStream(cstream).getLines().toSeq + private def readLines(in: InputStream): Seq[String] = { + Source.fromInputStream(in).getLines().toSeq } /** @@ -328,30 +220,14 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { var appEnded = false override def onJobStart(jobStart: SparkListenerJobStart) { - assertEventsExist(eventLogger, Seq[String]( - Utils.getFormattedClassName(SparkListenerApplicationStart), - Utils.getFormattedClassName(SparkListenerBlockManagerAdded), - Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) - )) jobStarted = true } override def onJobEnd(jobEnd: SparkListenerJobEnd) { - assertEventsExist(eventLogger, Seq[String]( - Utils.getFormattedClassName(SparkListenerJobStart), - Utils.getFormattedClassName(SparkListenerJobEnd), - Utils.getFormattedClassName(SparkListenerStageSubmitted), - Utils.getFormattedClassName(SparkListenerStageCompleted), - Utils.getFormattedClassName(SparkListenerTaskStart), - Utils.getFormattedClassName(SparkListenerTaskEnd) - )) jobEnded = true } override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { - assertEventsExist(eventLogger, Seq[String]( - Utils.getFormattedClassName(SparkListenerApplicationEnd) - )) appEnded = true } @@ -362,39 +238,6 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { } } - - /* -------------------------------------------------------- * - * Helper methods for validating state of the special files * - * -------------------------------------------------------- */ - - private def eventLogsExist(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isEventLogFile) - } - - private def sparkVersionExists(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isSparkVersionFile) - } - - private def compressionCodecExists(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isCompressionCodecFile) - } - - private def applicationCompleteExists(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isApplicationCompleteFile) - } - - private def assertSparkVersionIsValid(logFiles: Array[FileStatus]) { - val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isSparkVersionFile) - assert(file.isDefined) - assert(EventLoggingListener.parseSparkVersion(file.get) === SPARK_VERSION) - } - - private def assertCompressionCodecIsValid(logFiles: Array[FileStatus], compressionCodec: String) { - val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isCompressionCodecFile) - assert(file.isDefined) - assert(EventLoggingListener.parseCompressionCodec(file.get) === compressionCodec) - } - } diff --git a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala new file mode 100644 index 0000000000000..e8f461e2f56c9 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.util.Properties + +import org.scalatest.FunSuite + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} + +/** + * Tests that pools and the associated scheduling algorithms for FIFO and fair scheduling work + * correctly. + */ +class PoolSuite extends FunSuite with LocalSparkContext { + + def createTaskSetManager(stageId: Int, numTasks: Int, taskScheduler: TaskSchedulerImpl) + : TaskSetManager = { + val tasks = Array.tabulate[Task[_]](numTasks) { i => + new FakeTask(i, Nil) + } + new TaskSetManager(taskScheduler, new TaskSet(tasks, stageId, 0, 0, null), 0) + } + + def scheduleTaskAndVerifyId(taskId: Int, rootPool: Pool, expectedStageId: Int) { + val taskSetQueue = rootPool.getSortedTaskSetQueue + val nextTaskSetToSchedule = + taskSetQueue.find(t => (t.runningTasks + t.tasksSuccessful) < t.numTasks) + assert(nextTaskSetToSchedule.isDefined) + nextTaskSetToSchedule.get.addRunningTask(taskId) + assert(nextTaskSetToSchedule.get.stageId === expectedStageId) + } + + test("FIFO Scheduler Test") { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + + val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) + val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + val taskSetManager0 = createTaskSetManager(0, 2, taskScheduler) + val taskSetManager1 = createTaskSetManager(1, 2, taskScheduler) + val taskSetManager2 = createTaskSetManager(2, 2, taskScheduler) + schedulableBuilder.addTaskSetManager(taskSetManager0, null) + schedulableBuilder.addTaskSetManager(taskSetManager1, null) + schedulableBuilder.addTaskSetManager(taskSetManager2, null) + + scheduleTaskAndVerifyId(0, rootPool, 0) + scheduleTaskAndVerifyId(1, rootPool, 0) + scheduleTaskAndVerifyId(2, rootPool, 1) + scheduleTaskAndVerifyId(3, rootPool, 1) + scheduleTaskAndVerifyId(4, rootPool, 2) + scheduleTaskAndVerifyId(5, rootPool, 2) + } + + /** + * This test creates three scheduling pools, and creates task set managers in the first + * two scheduling pools. The test verifies that as tasks are scheduled, the fair scheduling + * algorithm properly orders the two scheduling pools. + */ + test("Fair Scheduler Test") { + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + val conf = new SparkConf().set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) + val taskScheduler = new TaskSchedulerImpl(sc) + + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) + schedulableBuilder.buildPools() + + // Ensure that the XML file was read in correctly. + assert(rootPool.getSchedulableByName("default") != null) + assert(rootPool.getSchedulableByName("1") != null) + assert(rootPool.getSchedulableByName("2") != null) + assert(rootPool.getSchedulableByName("3") != null) + assert(rootPool.getSchedulableByName("1").minShare === 2) + assert(rootPool.getSchedulableByName("1").weight === 1) + assert(rootPool.getSchedulableByName("2").minShare === 3) + assert(rootPool.getSchedulableByName("2").weight === 1) + assert(rootPool.getSchedulableByName("3").minShare === 0) + assert(rootPool.getSchedulableByName("3").weight === 1) + + val properties1 = new Properties() + properties1.setProperty("spark.scheduler.pool","1") + val properties2 = new Properties() + properties2.setProperty("spark.scheduler.pool","2") + + val taskSetManager10 = createTaskSetManager(0, 1, taskScheduler) + val taskSetManager11 = createTaskSetManager(1, 1, taskScheduler) + val taskSetManager12 = createTaskSetManager(2, 2, taskScheduler) + schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) + + val taskSetManager23 = createTaskSetManager(3, 2, taskScheduler) + val taskSetManager24 = createTaskSetManager(4, 2, taskScheduler) + schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) + schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) + + // Pool 1 share ratio: 0. Pool 2 share ratio: 0. 1 gets scheduled based on ordering of names. + scheduleTaskAndVerifyId(0, rootPool, 0) + // Pool 1 share ratio: 1/2. Pool 2 share ratio: 0. 2 gets scheduled because ratio is lower. + scheduleTaskAndVerifyId(1, rootPool, 3) + // Pool 1 share ratio: 1/2. Pool 2 share ratio: 1/3. 2 gets scheduled because ratio is lower. + scheduleTaskAndVerifyId(2, rootPool, 3) + // Pool 1 share ratio: 1/2. Pool 2 share ratio: 2/3. 1 gets scheduled because ratio is lower. + scheduleTaskAndVerifyId(3, rootPool, 1) + // Pool 1 share ratio: 1. Pool 2 share ratio: 2/3. 2 gets scheduled because ratio is lower. + scheduleTaskAndVerifyId(4, rootPool, 4) + // Neither pool is needy so ordering is based on number of running tasks. + // Pool 1 running tasks: 2, Pool 2 running tasks: 3. 1 gets scheduled because fewer running + // tasks. + scheduleTaskAndVerifyId(5, rootPool, 2) + // Pool 1 running tasks: 3, Pool 2 running tasks: 3. 1 gets scheduled because of naming + // ordering. + scheduleTaskAndVerifyId(6, rootPool, 2) + // Pool 1 running tasks: 4, Pool 2 running tasks: 3. 2 gets scheduled because fewer running + // tasks. + scheduleTaskAndVerifyId(7, rootPool, 4) + } + + test("Nested Pool Test") { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) + val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) + rootPool.addSchedulable(pool0) + rootPool.addSchedulable(pool1) + + val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) + val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) + pool0.addSchedulable(pool00) + pool0.addSchedulable(pool01) + + val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) + val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) + pool1.addSchedulable(pool10) + pool1.addSchedulable(pool11) + + val taskSetManager000 = createTaskSetManager(0, 5, taskScheduler) + val taskSetManager001 = createTaskSetManager(1, 5, taskScheduler) + pool00.addSchedulable(taskSetManager000) + pool00.addSchedulable(taskSetManager001) + + val taskSetManager010 = createTaskSetManager(2, 5, taskScheduler) + val taskSetManager011 = createTaskSetManager(3, 5, taskScheduler) + pool01.addSchedulable(taskSetManager010) + pool01.addSchedulable(taskSetManager011) + + val taskSetManager100 = createTaskSetManager(4, 5, taskScheduler) + val taskSetManager101 = createTaskSetManager(5, 5, taskScheduler) + pool10.addSchedulable(taskSetManager100) + pool10.addSchedulable(taskSetManager101) + + val taskSetManager110 = createTaskSetManager(6, 5, taskScheduler) + val taskSetManager111 = createTaskSetManager(7, 5, taskScheduler) + pool11.addSchedulable(taskSetManager110) + pool11.addSchedulable(taskSetManager111) + + scheduleTaskAndVerifyId(0, rootPool, 0) + scheduleTaskAndVerifyId(1, rootPool, 4) + scheduleTaskAndVerifyId(2, rootPool, 6) + scheduleTaskAndVerifyId(3, rootPool, 2) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 90bdfe07f61c9..7e360cc6082ec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -22,6 +22,7 @@ import java.io.{File, PrintWriter} import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.{SparkConf, SparkContext, SPARK_VERSION} import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec @@ -33,7 +34,6 @@ import org.apache.spark.util.{JsonProtocol, Utils} class ReplayListenerSuite extends FunSuite with BeforeAndAfter { private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) - private val allCompressionCodecs = CompressionCodec.ALL_COMPRESSION_CODECS private var testDir: File = _ before { @@ -45,13 +45,29 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { } test("Simple replay") { - testSimpleReplay() - } + val logFilePath = Utils.getFilePath(testDir, "events.txt") + val fstream = fileSystem.create(logFilePath) + val writer = new PrintWriter(fstream) + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, + 125L, "Mickey") + val applicationEnd = SparkListenerApplicationEnd(1000L) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + writer.close() - test("Simple replay with compression") { - allCompressionCodecs.foreach { codec => - testSimpleReplay(Some(codec)) + val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val logData = fileSystem.open(logFilePath) + val eventMonster = new EventMonster(conf) + try { + val replayer = new ReplayListenerBus() + replayer.addListener(eventMonster) + replayer.replay(logData, SPARK_VERSION) + } finally { + logData.close() } + assert(eventMonster.loggedEvents.size === 2) + assert(eventMonster.loggedEvents(0) === JsonProtocol.sparkEventToJson(applicationStart)) + assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) } // This assumes the correctness of EventLoggingListener @@ -61,7 +77,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { // This assumes the correctness of EventLoggingListener test("End-to-end replay with compression") { - allCompressionCodecs.foreach { codec => + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testApplicationReplay(Some(codec)) } } @@ -71,31 +87,6 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * Actual test logic * * ----------------- */ - /** - * Test simple replaying of events. - */ - private def testSimpleReplay(codecName: Option[String] = None) { - val logFilePath = Utils.getFilePath(testDir, "events.txt") - val codec = codecName.map(getCompressionCodec) - val fstream = fileSystem.create(logFilePath) - val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) - val writer = new PrintWriter(cstream) - val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, - 125L, "Mickey") - val applicationEnd = SparkListenerApplicationEnd(1000L) - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) - writer.close() - val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) - val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath, codecName) - val eventMonster = new EventMonster(conf) - replayer.addListener(eventMonster) - replayer.replay() - assert(eventMonster.loggedEvents.size === 2) - assert(eventMonster.loggedEvents(0) === JsonProtocol.sparkEventToJson(applicationStart)) - assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) - } - /** * Test end-to-end replaying of events. * @@ -105,6 +96,8 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { */ private def testApplicationReplay(codecName: Option[String] = None) { val logDirPath = Utils.getFilePath(testDir, "test-replay") + fileSystem.mkdirs(logDirPath) + val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) val sc = new SparkContext("local-cluster[2,1,512]", "Test replay", conf) @@ -116,22 +109,21 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { sc.stop() // Prepare information needed for replay - val codec = codecName.map(getCompressionCodec) val applications = fileSystem.listStatus(logDirPath) assert(applications != null && applications.size > 0) - val eventLogDir = applications.sortBy(_.getAccessTime).last - assert(eventLogDir.isDir) - val logFiles = fileSystem.listStatus(eventLogDir.getPath) - assert(logFiles != null && logFiles.size > 0) - val logFile = logFiles.find(_.getPath.getName.startsWith("EVENT_LOG_")) - assert(logFile.isDefined) - val logFilePath = logFile.get.getPath + val eventLog = applications.sortBy(_.getModificationTime).last + assert(!eventLog.isDir) // Replay events - val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) + val (logData, version) = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) val eventMonster = new EventMonster(conf) - replayer.addListener(eventMonster) - replayer.replay() + try { + val replayer = new ReplayListenerBus() + replayer.addListener(eventMonster) + replayer.replay(logData, version) + } finally { + logData.close() + } // Verify the same events are replayed in the same order assert(sc.eventLogger.isDefined) @@ -154,7 +146,9 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { */ private class EventMonster(conf: SparkConf) extends EventLoggingListener("test", "testdir", conf) { - logger.close() + + override def start() { } + } private def getCompressionCodec(codecName: String) = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index b276343cb412c..24f41bf8cccda 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -26,9 +26,10 @@ import org.scalatest.Matchers import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.ResetSystemProperties -class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers - with BeforeAndAfter with BeforeAndAfterAll { +class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers with BeforeAndAfter + with BeforeAndAfterAll with ResetSystemProperties { /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 @@ -37,10 +38,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers sc = new SparkContext("local", "SparkListenerSuite") } - override def afterAll() { - System.clearProperty("spark.akka.frameSize") - } - test("basic creation and shutdown of LiveListenerBus") { val counter = new BasicJobCounter val bus = new LiveListenerBus diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 5768a3a733f00..e3a3803e6483a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -19,9 +19,14 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.control.NonFatal -import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv} import org.apache.spark.storage.TaskResultBlockId /** @@ -34,6 +39,8 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule extends TaskResultGetter(sparkEnv, scheduler) { var removedResult = false + @volatile var removeBlockSuccessfully = false + override def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { if (!removedResult) { @@ -42,6 +49,15 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule serializer.get().deserialize[TaskResult[_]](serializedData) match { case IndirectTaskResult(blockId, size) => sparkEnv.blockManager.master.removeBlock(blockId) + // removeBlock is asynchronous. Need to wait it's removed successfully + try { + eventually(timeout(3 seconds), interval(200 milliseconds)) { + assert(!sparkEnv.blockManager.master.contains(blockId)) + } + removeBlockSuccessfully = true + } catch { + case NonFatal(e) => removeBlockSuccessfully = false + } case directResult: DirectTaskResult[_] => taskSetManager.abort("Internal error: expect only indirect results") } @@ -55,27 +71,20 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule /** * Tests related to handling task results (both direct and indirect). */ -class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll - with LocalSparkContext { +class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - override def beforeAll { - // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small - // as we can make it) so the tests don't take too long. - System.setProperty("spark.akka.frameSize", "1") - } - - override def afterAll { - System.clearProperty("spark.akka.frameSize") - } + // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small + // as we can make it) so the tests don't take too long. + def conf: SparkConf = new SparkConf().set("spark.akka.frameSize", "1") test("handling results smaller than Akka frame size") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) assert(result === 2) } test("handling results larger than Akka frame size") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) @@ -89,7 +98,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA test("task retried if result missing from block manager") { // Set the maximum number of task failures to > 0, so that the task set isn't aborted // after the result is missing. - sc = new SparkContext("local[1,2]", "test") + sc = new SparkContext("local[1,2]", "test", conf) // If this test hangs, it's probably because no resource offers were made after the task // failed. val scheduler: TaskSchedulerImpl = sc.taskScheduler match { @@ -99,10 +108,12 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA assert(false, "Expect local cluster to use TaskSchedulerImpl") throw new ClassCastException } - scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) + val resultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) + scheduler.taskResultGetter = resultGetter val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) + assert(resultGetter.removeBlockSuccessfully) assert(result === 1.to(akkaFrameSize).toArray) // Make sure two tasks were run (one failed one, and a second retried one). diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 7532da88c6065..8874cf00e9993 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -30,238 +30,8 @@ class FakeSchedulerBackend extends SchedulerBackend { def defaultParallelism() = 1 } -class FakeTaskSetManager( - initPriority: Int, - initStageId: Int, - initNumTasks: Int, - taskScheduler: TaskSchedulerImpl, - taskSet: TaskSet) - extends TaskSetManager(taskScheduler, taskSet, 0) { - - parent = null - weight = 1 - minShare = 2 - priority = initPriority - stageId = initStageId - name = "TaskSet_"+stageId - override val numTasks = initNumTasks - tasksSuccessful = 0 - - var numRunningTasks = 0 - override def runningTasks = numRunningTasks - - def increaseRunningTasks(taskNum: Int) { - numRunningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } - } - - def decreaseRunningTasks(taskNum: Int) { - numRunningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) - } - } - - override def addSchedulable(schedulable: Schedulable) { - } - - override def removeSchedulable(schedulable: Schedulable) { - } - - override def getSchedulableByName(name: String): Schedulable = { - null - } - - override def executorLost(executorId: String, host: String): Unit = { - } - - override def resourceOffer( - execId: String, - host: String, - maxLocality: TaskLocality.TaskLocality) - : Option[TaskDescription] = - { - if (tasksSuccessful + numRunningTasks < numTasks) { - increaseRunningTasks(1) - Some(new TaskDescription(0, execId, "task 0:0", 0, null)) - } else { - None - } - } - - override def checkSpeculatableTasks(): Boolean = { - true - } - - def taskFinished() { - decreaseRunningTasks(1) - tasksSuccessful +=1 - if (tasksSuccessful == numTasks) { - parent.removeSchedulable(this) - } - } - - def abort() { - decreaseRunningTasks(numRunningTasks) - parent.removeSchedulable(this) - } -} - class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Logging { - def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskSchedulerImpl, - taskSet: TaskSet): FakeTaskSetManager = { - new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet) - } - - def resourceOffer(rootPool: Pool): Int = { - val taskSetQueue = rootPool.getSortedTaskSetQueue - /* Just for Test*/ - for (manager <- taskSetQueue) { - logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( - manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) - } - for (taskSet <- taskSetQueue) { - taskSet.resourceOffer("execId_1", "hostname_1", TaskLocality.ANY) match { - case Some(task) => - return taskSet.stageId - case None => {} - } - } - -1 - } - - def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { - assert(resourceOffer(rootPool) === expectedTaskSetId) - } - - test("FIFO Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - val taskSet = FakeTask.createTaskSet(1) - - val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) - val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) - schedulableBuilder.buildPools() - - val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, taskScheduler, taskSet) - val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, taskScheduler, taskSet) - val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, taskScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager0, null) - schedulableBuilder.addTaskSetManager(taskSetManager1, null) - schedulableBuilder.addTaskSetManager(taskSetManager2, null) - - checkTaskSetId(rootPool, 0) - resourceOffer(rootPool) - checkTaskSetId(rootPool, 1) - resourceOffer(rootPool) - taskSetManager1.abort() - checkTaskSetId(rootPool, 2) - } - - test("Fair Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - val taskSet = FakeTask.createTaskSet(1) - - val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) - schedulableBuilder.buildPools() - - assert(rootPool.getSchedulableByName("default") != null) - assert(rootPool.getSchedulableByName("1") != null) - assert(rootPool.getSchedulableByName("2") != null) - assert(rootPool.getSchedulableByName("3") != null) - assert(rootPool.getSchedulableByName("1").minShare === 2) - assert(rootPool.getSchedulableByName("1").weight === 1) - assert(rootPool.getSchedulableByName("2").minShare === 3) - assert(rootPool.getSchedulableByName("2").weight === 1) - assert(rootPool.getSchedulableByName("3").minShare === 0) - assert(rootPool.getSchedulableByName("3").weight === 1) - - val properties1 = new Properties() - properties1.setProperty("spark.scheduler.pool","1") - val properties2 = new Properties() - properties2.setProperty("spark.scheduler.pool","2") - - val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, taskScheduler, taskSet) - val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, taskScheduler, taskSet) - val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, taskScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) - schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) - schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - - val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, taskScheduler, taskSet) - val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, taskScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) - schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) - - checkTaskSetId(rootPool, 0) - checkTaskSetId(rootPool, 3) - checkTaskSetId(rootPool, 3) - checkTaskSetId(rootPool, 1) - checkTaskSetId(rootPool, 4) - checkTaskSetId(rootPool, 2) - checkTaskSetId(rootPool, 2) - checkTaskSetId(rootPool, 4) - - taskSetManager12.taskFinished() - assert(rootPool.getSchedulableByName("1").runningTasks === 3) - taskSetManager24.abort() - assert(rootPool.getSchedulableByName("2").runningTasks === 2) - } - - test("Nested Pool Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - val taskSet = FakeTask.createTaskSet(1) - - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) - val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) - rootPool.addSchedulable(pool0) - rootPool.addSchedulable(pool1) - - val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) - val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) - pool0.addSchedulable(pool00) - pool0.addSchedulable(pool01) - - val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) - val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) - pool1.addSchedulable(pool10) - pool1.addSchedulable(pool11) - - val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, taskScheduler, taskSet) - val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, taskScheduler, taskSet) - pool00.addSchedulable(taskSetManager000) - pool00.addSchedulable(taskSetManager001) - - val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, taskScheduler, taskSet) - val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, taskScheduler, taskSet) - pool01.addSchedulable(taskSetManager010) - pool01.addSchedulable(taskSetManager011) - - val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, taskScheduler, taskSet) - val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, taskScheduler, taskSet) - pool10.addSchedulable(taskSetManager100) - pool10.addSchedulable(taskSetManager101) - - val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, taskScheduler, taskSet) - val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, taskScheduler, taskSet) - pool11.addSchedulable(taskSetManager110) - pool11.addSchedulable(taskSetManager111) - - checkTaskSetId(rootPool, 0) - checkTaskSetId(rootPool, 4) - checkTaskSetId(rootPool, 6) - checkTaskSetId(rootPool, 2) - } - test("Scheduler does not always schedule tasks on the same workers") { sc = new SparkContext("local", "TaskSchedulerImplSuite") val taskScheduler = new TaskSchedulerImpl(sc) @@ -305,7 +75,6 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} override def executorAdded(execId: String, host: String) {} } - taskScheduler.setDAGScheduler(dagScheduler) // Give zero core offers. Should not generate any tasks val zeroCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", 0), new WorkerOffer("executor1", "host1", 0)) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 5554efbcbadf8..ffe6f039145ea 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -33,7 +33,7 @@ import akka.util.Timeout import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} +import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ @@ -44,18 +44,17 @@ import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat -import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import org.apache.spark.util._ -class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter - with PrivateMethodTester { +class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach + with PrivateMethodTester with ResetSystemProperties { private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null var master: BlockManagerMaster = null - var oldArch: String = null conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) val mapOutputTracker = new MapOutputTrackerMaster(conf) @@ -79,13 +78,13 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter manager } - before { + override def beforeEach(): Unit = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem( "test", "localhost", 0, conf = conf, securityManager = securityMgr) this.actorSystem = actorSystem // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") + System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") conf.set("spark.driver.port", boundPort.toString) @@ -100,7 +99,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter SizeEstimator invokePrivate initialize() } - after { + override def afterEach(): Unit = { if (store != null) { store.stop() store = null @@ -113,14 +112,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter actorSystem.awaitTermination() actorSystem = null master = null - - if (oldArch != null) { - conf.set("os.arch", oldArch) - } else { - System.clearProperty("os.arch") - } - - System.clearProperty("spark.test.useCompressedOops") } test("StorageLevel object caching") { diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 7bca1711ae226..6bbf72e929dcb 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.BlockManagerId /** * Test the AkkaUtils with various security settings. */ -class AkkaUtilsSuite extends FunSuite with LocalSparkContext { +class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { test("remote fetch security bad password") { val conf = new SparkConf diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala deleted file mode 100644 index 72466a3aa1130..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala +++ /dev/null @@ -1,169 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import java.io.{File, IOException} - -import scala.io.Source - -import org.apache.hadoop.fs.Path -import org.scalatest.{BeforeAndAfter, FunSuite} - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.io.CompressionCodec - -/** - * Test writing files through the FileLogger. - */ -class FileLoggerSuite extends FunSuite with BeforeAndAfter { - private val fileSystem = Utils.getHadoopFileSystem("/", - SparkHadoopUtil.get.newConfiguration(new SparkConf())) - private val allCompressionCodecs = Seq[String]( - "org.apache.spark.io.LZFCompressionCodec", - "org.apache.spark.io.SnappyCompressionCodec" - ) - private var testDir: File = _ - private var logDirPath: Path = _ - private var logDirPathString: String = _ - - before { - testDir = Utils.createTempDir() - logDirPath = Utils.getFilePath(testDir, "test-file-logger") - logDirPathString = logDirPath.toString - } - - after { - Utils.deleteRecursively(testDir) - } - - test("Simple logging") { - testSingleFile() - } - - test ("Simple logging with compression") { - allCompressionCodecs.foreach { codec => - testSingleFile(Some(codec)) - } - } - - test("Logging multiple files") { - testMultipleFiles() - } - - test("Logging multiple files with compression") { - allCompressionCodecs.foreach { codec => - testMultipleFiles(Some(codec)) - } - } - - test("Logging when directory already exists") { - // Create the logging directory multiple times - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() - - // If overwrite is not enabled, an exception should be thrown - intercept[IOException] { - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = false).start() - } - } - - - /* ----------------- * - * Actual test logic * - * ----------------- */ - - /** - * Test logging to a single file. - */ - private def testSingleFile(codecName: Option[String] = None) { - val conf = getLoggingConf(codecName) - val codec = codecName.map { c => CompressionCodec.createCodec(conf) } - val logger = - if (codecName.isDefined) { - new FileLogger(logDirPathString, conf, compress = true) - } else { - new FileLogger(logDirPathString, conf) - } - logger.start() - assert(fileSystem.exists(logDirPath)) - assert(fileSystem.getFileStatus(logDirPath).isDir) - assert(fileSystem.listStatus(logDirPath).size === 0) - - logger.newFile() - val files = fileSystem.listStatus(logDirPath) - assert(files.size === 1) - val firstFile = files.head - val firstFilePath = firstFile.getPath - - logger.log("hello") - logger.flush() - assert(readFileContent(firstFilePath, codec) === "hello") - - logger.log(" world") - logger.close() - assert(readFileContent(firstFilePath, codec) === "hello world") - } - - /** - * Test logging to multiple files. - */ - private def testMultipleFiles(codecName: Option[String] = None) { - val conf = getLoggingConf(codecName) - val codec = codecName.map { c => CompressionCodec.createCodec(conf) } - val logger = - if (codecName.isDefined) { - new FileLogger(logDirPathString, conf, compress = true) - } else { - new FileLogger(logDirPathString, conf) - } - logger.start() - logger.newFile("Jean_Valjean") - logger.logLine("Who am I?") - logger.logLine("Destiny?") - logger.newFile("John_Valjohn") - logger.logLine("One") - logger.logLine("Two three...") - logger.newFile("Wolverine") - logger.logLine("There was a time") - logger.logLine("A time when our enemies knew honor.") - logger.close() - assert(readFileContent(new Path(logDirPath, "Jean_Valjean"), codec) === "Who am I?\nDestiny?") - assert(readFileContent(new Path(logDirPath, "John_Valjohn"), codec) === "One\nTwo three...") - assert(readFileContent(new Path(logDirPath, "Wolverine"), codec) === - "There was a time\nA time when our enemies knew honor.") - } - - /** - * Read the content of the file specified by the given path. - * If a compression codec is specified, use it to read the file. - */ - private def readFileContent(logPath: Path, codec: Option[CompressionCodec] = None): String = { - val fstream = fileSystem.open(logPath) - val cstream = codec.map(_.compressedInputStream(fstream)).getOrElse(fstream) - Source.fromInputStream(cstream).getLines().mkString("\n") - } - - private def getLoggingConf(codecName: Option[String]) = { - val conf = new SparkConf - codecName.foreach { c => conf.set("spark.io.compression.codec", c) } - conf - } - -} diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index c0fe9b80a8e44..25693ee866487 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -280,7 +280,7 @@ class JsonProtocolSuite extends FunSuite { private def testBlockManagerId(id: BlockManagerId) { val newId = JsonProtocol.blockManagerIdFromJson(JsonProtocol.blockManagerIdToJson(id)) - assertEquals(id, newId) + assert(id === newId) } private def testTaskInfo(info: TaskInfo) { @@ -335,22 +335,8 @@ class JsonProtocolSuite extends FunSuite { assertEquals(e1.jobResult, e2.jobResult) case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) => assertEquals(e1.environmentDetails, e2.environmentDetails) - case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) => - assert(e1.maxMem === e2.maxMem) - assert(e1.time === e2.time) - assertEquals(e1.blockManagerId, e2.blockManagerId) - case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => - assert(e1.time === e2.time) - assertEquals(e1.blockManagerId, e2.blockManagerId) - case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => - assert(e1.rddId == e2.rddId) - case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => - assert(e1.appName == e2.appName) - assert(e1.time == e2.time) - assert(e1.sparkUser == e2.sparkUser) - case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => - assert(e1.time == e2.time) - case (SparkListenerShutdown, SparkListenerShutdown) => + case (e1, e2) => + assert(e1 === e2) case _ => fail("Events don't match in types!") } } @@ -435,16 +421,6 @@ class JsonProtocolSuite extends FunSuite { assert(metrics1.bytesRead === metrics2.bytesRead) } - private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { - if (bm1 == null || bm2 == null) { - assert(bm1 === bm2) - } else { - assert(bm1.executorId === bm2.executorId) - assert(bm1.host === bm2.host) - assert(bm1.port === bm2.port) - } - } - private def assertEquals(result1: JobResult, result2: JobResult) { (result1, result2) match { case (JobSucceeded, JobSucceeded) => @@ -462,7 +438,7 @@ class JsonProtocolSuite extends FunSuite { assert(r1.shuffleId === r2.shuffleId) assert(r1.mapId === r2.mapId) assert(r1.reduceId === r2.reduceId) - assertEquals(r1.bmAddress, r2.bmAddress) + assert(r1.bmAddress === r2.bmAddress) assert(r1.message === r2.message) case (r1: ExceptionFailure, r2: ExceptionFailure) => assert(r1.className === r2.className) diff --git a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala new file mode 100644 index 0000000000000..d4b92f33dd9e6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala @@ -0,0 +1,57 @@ +/* + * 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.util + +import java.util.Properties + +import org.scalatest.{BeforeAndAfterEach, Suite} + +/** + * Mixin for automatically resetting system properties that are modified in ScalaTest tests. + * This resets the properties after each individual test. + * + * The order in which fixtures are mixed in affects the order in which they are invoked by tests. + * If we have a suite `MySuite extends FunSuite with Foo with Bar`, then + * Bar's `super` is Foo, so Bar's beforeEach() will and afterEach() methods will be invoked first + * by the rest runner. + * + * This means that ResetSystemProperties should appear as the last trait in test suites that it's + * mixed into in order to ensure that the system properties snapshot occurs as early as possible. + * ResetSystemProperties calls super.afterEach() before performing its own cleanup, ensuring that + * the old properties are restored as late as possible. + * + * See the "Composing fixtures by stacking traits" section at + * http://www.scalatest.org/user_guide/sharing_fixtures for more details about this pattern. + */ +private[spark] trait ResetSystemProperties extends BeforeAndAfterEach { this: Suite => + var oldProperties: Properties = null + + override def beforeEach(): Unit = { + oldProperties = new Properties(System.getProperties) + super.beforeEach() + } + + override def afterEach(): Unit = { + try { + super.afterEach() + } finally { + System.setProperties(oldProperties) + oldProperties = null + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 0ea2d13a83505..7424c2e91d4f2 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.util -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite -import org.scalatest.PrivateMethodTester +import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite, PrivateMethodTester} class DummyClass1 {} @@ -46,20 +44,12 @@ class DummyString(val arr: Array[Char]) { } class SizeEstimatorSuite - extends FunSuite with BeforeAndAfterAll with PrivateMethodTester { + extends FunSuite with BeforeAndAfterEach with PrivateMethodTester with ResetSystemProperties { - var oldArch: String = _ - var oldOops: String = _ - - override def beforeAll() { + override def beforeEach() { // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") - oldOops = System.setProperty("spark.test.useCompressedOops", "true") - } - - override def afterAll() { - resetOrClear("os.arch", oldArch) - resetOrClear("spark.test.useCompressedOops", oldOops) + System.setProperty("os.arch", "amd64") + System.setProperty("spark.test.useCompressedOops", "true") } test("simple classes") { @@ -122,7 +112,7 @@ class SizeEstimatorSuite } test("32-bit arch") { - val arch = System.setProperty("os.arch", "x86") + System.setProperty("os.arch", "x86") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -131,14 +121,13 @@ class SizeEstimatorSuite assertResult(48)(SizeEstimator.estimate(DummyString("a"))) assertResult(48)(SizeEstimator.estimate(DummyString("ab"))) assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) - resetOrClear("os.arch", arch) } // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("64-bit arch with no compressed oops") { - val arch = System.setProperty("os.arch", "amd64") - val oops = System.setProperty("spark.test.useCompressedOops", "false") + System.setProperty("os.arch", "amd64") + System.setProperty("spark.test.useCompressedOops", "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -146,16 +135,5 @@ class SizeEstimatorSuite assertResult(64)(SizeEstimator.estimate(DummyString("a"))) assertResult(64)(SizeEstimator.estimate(DummyString("ab"))) assertResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) - - resetOrClear("os.arch", arch) - resetOrClear("spark.test.useCompressedOops", oops) - } - - def resetOrClear(prop: String, oldValue: String) { - if (oldValue != null) { - System.setProperty(prop, oldValue) - } else { - System.clearProperty(prop) - } } } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index f9d4bea823f7c..4544382094f96 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkConf -class UtilsSuite extends FunSuite { +class UtilsSuite extends FunSuite with ResetSystemProperties { test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 3b89aaba29609..b1b8cb44e098b 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -87,8 +87,8 @@ if [[ ! "$@" =~ --package-only ]]; then git commit -a -m "Preparing development version $next_ver" git push origin $GIT_TAG git push origin HEAD:$GIT_BRANCH - git checkout -f $GIT_TAG - + git checkout -f $GIT_TAG + # Using Nexus API documented here: # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API echo "Creating Nexus staging repository" @@ -106,7 +106,7 @@ if [[ ! "$@" =~ --package-only ]]; then clean install ./dev/change-version-to-2.11.sh - + mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dscala-2.11 -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install @@ -174,7 +174,7 @@ make_binary_release() { NAME=$1 FLAGS=$2 cp -r spark spark-$RELEASE_VERSION-bin-$NAME - + cd spark-$RELEASE_VERSION-bin-$NAME # TODO There should probably be a flag to make-distribution to allow 2.11 support @@ -219,7 +219,7 @@ scp spark-* \ # Docs cd spark -sbt/sbt clean +build/sbt clean cd docs # Compile docs with Java 7 to use nicer format JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index a3b78a3eac6d0..8aaa250bd7e29 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -26,75 +26,103 @@ # You must set the following before use! JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") -START_COMMIT = os.environ.get("START_COMMIT", "37b100") -END_COMMIT = os.environ.get("END_COMMIT", "3693ae") - -# If commit range is not specified, prompt the user to provide it -if not START_COMMIT or not END_COMMIT: - print "A commit range is required to proceed." - if not START_COMMIT: - START_COMMIT = raw_input("Please specify starting commit hash (inclusive): ") - if not END_COMMIT: - END_COMMIT = raw_input("Please specify ending commit hash (non-inclusive): ") - -# Verify provided arguments -start_commit_line = get_one_line(START_COMMIT) -end_commit_line = get_one_line(END_COMMIT) -num_commits = num_commits_in_range(START_COMMIT, END_COMMIT) -if not start_commit_line: sys.exit("Start commit %s not found!" % START_COMMIT) -if not end_commit_line: sys.exit("End commit %s not found!" % END_COMMIT) -if num_commits == 0: - sys.exit("There are no commits in the provided range [%s, %s)" % (START_COMMIT, END_COMMIT)) +RELEASE_TAG = os.environ.get("RELEASE_TAG", "v1.2.0-rc2") +PREVIOUS_RELEASE_TAG = os.environ.get("PREVIOUS_RELEASE_TAG", "v1.1.0") + +# If the release tags are not provided, prompt the user to provide them +while not tag_exists(RELEASE_TAG): + RELEASE_TAG = raw_input("Please provide a valid release tag: ") +while not tag_exists(PREVIOUS_RELEASE_TAG): + print "Please specify the previous release tag." + PREVIOUS_RELEASE_TAG = raw_input(\ + "For instance, if you are releasing v1.2.0, you should specify v1.1.0: ") + +# Gather commits found in the new tag but not in the old tag. +# This filters commits based on both the git hash and the PR number. +# If either is present in the old tag, then we ignore the commit. +print "Gathering new commits between tags %s and %s" % (PREVIOUS_RELEASE_TAG, RELEASE_TAG) +release_commits = get_commits(RELEASE_TAG) +previous_release_commits = get_commits(PREVIOUS_RELEASE_TAG) +previous_release_hashes = set() +previous_release_prs = set() +for old_commit in previous_release_commits: + previous_release_hashes.add(old_commit.get_hash()) + if old_commit.get_pr_number(): + previous_release_prs.add(old_commit.get_pr_number()) +new_commits = [] +for this_commit in release_commits: + this_hash = this_commit.get_hash() + this_pr_number = this_commit.get_pr_number() + if this_hash in previous_release_hashes: + continue + if this_pr_number and this_pr_number in previous_release_prs: + continue + new_commits.append(this_commit) +if not new_commits: + sys.exit("There are no new commits between %s and %s!" % (PREVIOUS_RELEASE_TAG, RELEASE_TAG)) + +# Prompt the user for confirmation that the commit range is correct print "\n==================================================================================" print "JIRA server: %s" % JIRA_API_BASE -print "Start commit (inclusive): %s" % start_commit_line -print "End commit (non-inclusive): %s" % end_commit_line -print "Number of commits in this range: %s" % num_commits +print "Release tag: %s" % RELEASE_TAG +print "Previous release tag: %s" % PREVIOUS_RELEASE_TAG +print "Number of commits in this range: %s" % len(new_commits) print -response = raw_input("Is this correct? [Y/n] ") -if response.lower() != "y" and response: - sys.exit("Ok, exiting") +def print_indented(_list): + for x in _list: print " %s" % x +if yesOrNoPrompt("Show all commits?"): + print_indented(new_commits) print "==================================================================================\n" - -# Find all commits within this range -print "Gathering commits within range [%s..%s)" % (START_COMMIT, END_COMMIT) -commits = get_one_line_commits(START_COMMIT, END_COMMIT) -if not commits: sys.exit("Error: No commits found within this range!") -commits = commits.split("\n") +if not yesOrNoPrompt("Does this look correct?"): + sys.exit("Ok, exiting") # Filter out special commits releases = [] +maintenance = [] reverts = [] nojiras = [] filtered_commits = [] -def is_release(commit): - return re.findall("\[release\]", commit.lower()) or\ - "maven-release-plugin" in commit or "CHANGES.txt" in commit -def has_no_jira(commit): - return not re.findall("SPARK-[0-9]+", commit.upper()) -def is_revert(commit): - return "revert" in commit.lower() -def is_docs(commit): - return re.findall("docs*", commit.lower()) or "programming guide" in commit.lower() -for c in commits: - if not c: continue - elif is_release(c): releases.append(c) - elif is_revert(c): reverts.append(c) - elif is_docs(c): filtered_commits.append(c) # docs may not have JIRA numbers - elif has_no_jira(c): nojiras.append(c) +def is_release(commit_title): + return re.findall("\[release\]", commit_title.lower()) or\ + "preparing spark release" in commit_title.lower() or\ + "preparing development version" in commit_title.lower() or\ + "CHANGES.txt" in commit_title +def is_maintenance(commit_title): + return "maintenance" in commit_title.lower() or\ + "manually close" in commit_title.lower() +def has_no_jira(commit_title): + return not re.findall("SPARK-[0-9]+", commit_title.upper()) +def is_revert(commit_title): + return "revert" in commit_title.lower() +def is_docs(commit_title): + return re.findall("docs*", commit_title.lower()) or\ + "programming guide" in commit_title.lower() +for c in new_commits: + t = c.get_title() + if not t: continue + elif is_release(t): releases.append(c) + elif is_maintenance(t): maintenance.append(c) + elif is_revert(t): reverts.append(c) + elif is_docs(t): filtered_commits.append(c) # docs may not have JIRA numbers + elif has_no_jira(t): nojiras.append(c) else: filtered_commits.append(c) # Warn against ignored commits -def print_indented(_list): - for x in _list: print " %s" % x -if releases or reverts or nojiras: +if releases or maintenance or reverts or nojiras: print "\n==================================================================================" - if releases: print "Releases (%d)" % len(releases); print_indented(releases) - if reverts: print "Reverts (%d)" % len(reverts); print_indented(reverts) - if nojiras: print "No JIRA (%d)" % len(nojiras); print_indented(nojiras) + if releases: print "Found %d release commits" % len(releases) + if maintenance: print "Found %d maintenance commits" % len(maintenance) + if reverts: print "Found %d revert commits" % len(reverts) + if nojiras: print "Found %d commits with no JIRA" % len(nojiras) + print "* Warning: these commits will be ignored.\n" + if yesOrNoPrompt("Show ignored commits?"): + if releases: print "Release (%d)" % len(releases); print_indented(releases) + if maintenance: print "Maintenance (%d)" % len(maintenance); print_indented(maintenance) + if reverts: print "Revert (%d)" % len(reverts); print_indented(reverts) + if nojiras: print "No JIRA (%d)" % len(nojiras); print_indented(nojiras) print "==================== Warning: the above commits will be ignored ==================\n" -response = raw_input("%d commits left to process. Ok to proceed? [Y/n] " % len(filtered_commits)) -if response.lower() != "y" and response: +prompt_msg = "%d commits left to process after filtering. Ok to proceed?" % len(filtered_commits) +if not yesOrNoPrompt(prompt_msg): sys.exit("Ok, exiting.") # Keep track of warnings to tell the user at the end @@ -123,10 +151,11 @@ def print_indented(_list): jira_client = JIRA(options = jira_options) print "\n=========================== Compiling contributor list ===========================" for commit in filtered_commits: - commit_hash = re.findall("^[a-z0-9]+", commit)[0] - issues = re.findall("SPARK-[0-9]+", commit.upper()) - author = get_author(commit_hash) - author = unidecode.unidecode(unicode(author, "UTF-8")).strip() # guard against special characters + _hash = commit.get_hash() + title = commit.get_title() + issues = re.findall("SPARK-[0-9]+", title.upper()) + author = commit.get_author() + date = get_date(_hash) # If the author name is invalid, keep track of it along # with all associated issues so we can translate it later if is_valid_author(author): @@ -136,9 +165,8 @@ def print_indented(_list): invalid_authors[author] = set() for issue in issues: invalid_authors[author].add(issue) - date = get_date(commit_hash) - # Parse components from the commit message, if any - commit_components = find_components(commit, commit_hash) + # Parse components from the commit title, if any + commit_components = find_components(title, _hash) # Populate or merge an issue into author_info[author] def populate(issue_type, components): components = components or [CORE_COMPONENT] # assume core if no components provided @@ -153,20 +181,20 @@ def populate(issue_type, components): jira_issue = jira_client.issue(issue) jira_type = jira_issue.fields.issuetype.name jira_type = translate_issue_type(jira_type, issue, warnings) - jira_components = [translate_component(c.name, commit_hash, warnings)\ - for c in jira_issue.fields.components] + jira_components = [translate_component(c.name, _hash, warnings)\ + for c in jira_issue.fields.components] all_components = set(jira_components + commit_components) populate(jira_type, all_components) # For docs without an associated JIRA, manually add it ourselves - if is_docs(commit) and not issues: + if is_docs(title) and not issues: populate("documentation", commit_components) - print " Processed commit %s authored by %s on %s" % (commit_hash, author, date) + print " Processed commit %s authored by %s on %s" % (_hash, author, date) print "==================================================================================\n" # Write to contributors file ordered by author names -# Each line takes the format "Author name - semi-colon delimited contributions" -# e.g. Andrew Or - Bug fixes in Windows, Core, and Web UI; improvements in Core -# e.g. Tathagata Das - Bug fixes and new features in Streaming +# Each line takes the format " * Author name -- semi-colon delimited contributions" +# e.g. * Andrew Or -- Bug fixes in Windows, Core, and Web UI; improvements in Core +# e.g. * Tathagata Das -- Bug fixes and new features in Streaming contributors_file = open(contributors_file_name, "w") authors = author_info.keys() authors.sort() @@ -185,7 +213,7 @@ def populate(issue_type, components): # e.g. Bug fixes in MLlib, Core, and Streaming; documentation in YARN else: contributions = ["%s in %s" % (issue_type, nice_join(comps)) \ - for issue_type, comps in author_info[author].items()] + for issue_type, comps in author_info[author].items()] contribution = "; ".join(contributions) # Do not use python's capitalize() on the whole string to preserve case assert contribution @@ -195,7 +223,7 @@ def populate(issue_type, components): # E.g. andrewor14/SPARK-3425/SPARK-1157/SPARK-6672 if author in invalid_authors and invalid_authors[author]: author = author + "/" + "/".join(invalid_authors[author]) - line = "%s - %s" % (author, contribution) + line = " * %s -- %s" % (author, contribution) contributors_file.write(line + "\n") contributors_file.close() print "Contributors list is successfully written to %s!" % contributors_file_name diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations new file mode 100644 index 0000000000000..b74e4ee8a330b --- /dev/null +++ b/dev/create-release/known_translations @@ -0,0 +1,59 @@ +# This is a mapping of names to be translated through translate-contributors.py +# The format expected on each line should be: - +CodingCat - Nan Zhu +CrazyJvm - Chao Chen +EugenCepoi - Eugen Cepoi +GraceH - Jie Huang +JerryLead - Lijie Xu +Leolh - Liu Hao +Lewuathe - Kai Sasaki +RongGu - Rong Gu +Shiti - Shiti Saxena +Victsm - Min Shen +WangTaoTheTonic - Wang Tao +XuTingjun - Tingjun Xu +YanTangZhai - Yantang Zhai +alexdebrie - Alex DeBrie +alokito - Alok Saldanha +anantasty - Anant Asthana +andrewor14 - Andrew Or +aniketbhatnagar - Aniket Bhatnagar +arahuja - Arun Ahuja +brkyvz - Burak Yavuz +chesterxgchen - Chester Chen +chiragaggarwal - Chirag Aggarwal +chouqin - Qiping Li +cocoatomo - Tomohiko K. +coderfi - Fairiz Azizi +coderxiang - Shuo Xiang +davies - Davies Liu +epahomov - Egor Pahomov +falaki - Hossein Falaki +freeman-lab - Jeremy Freeman +industrial-sloth - Jascha Swisher +jackylk - Jacky Li +jayunit100 - Jay Vyas +jerryshao - Saisai Shao +jkbradley - Joseph Bradley +lianhuiwang - Lianhui Wang +lirui-intel - Rui Li +luluorta - Lu Lu +luogankun - Gankun Luo +maji2014 - Derek Ma +mccheah - Matthew Cheah +mengxr - Xiangrui Meng +nartz - Nathan Artz +odedz - Oded Zimerman +ravipesala - Ravindra Pesala +roxchkplusony - Victor Tso +scwf - Wang Fei +shimingfei - Shiming Fei +surq - Surong Quan +suyanNone - Su Yan +tedyu - Ted Yu +tigerquoll - Dale Richardson +wangxiaojing - Xiaojing Wang +watermen - Yadong Qi +witgo - Guoqiang Li +xinyunh - Xinyun Huang +zsxwing - Shixiong Zhu diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 76a10c32886d4..26221b270394e 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -19,6 +19,7 @@ # This file contains helper methods used in creating a release. import re +import sys from subprocess import Popen, PIPE try: @@ -47,20 +48,85 @@ # Contributors list file name contributors_file_name = "contributors.txt" +# Prompt the user to answer yes or no until they do so +def yesOrNoPrompt(msg): + response = raw_input("%s [y/n]: " % msg) + while response != "y" and response != "n": + return yesOrNoPrompt(msg) + return response == "y" + # Utility functions run git commands (written with Git 1.8.5) def run_cmd(cmd): return Popen(cmd, stdout=PIPE).communicate()[0] -def get_author(commit_hash): - return run_cmd(["git", "show", "--quiet", "--pretty=format:%an", commit_hash]) +def run_cmd_error(cmd): return Popen(cmd, stdout=PIPE, stderr=PIPE).communicate()[1] def get_date(commit_hash): return run_cmd(["git", "show", "--quiet", "--pretty=format:%cd", commit_hash]) -def get_one_line(commit_hash): - return run_cmd(["git", "show", "--quiet", "--pretty=format:\"%h %cd %s\"", commit_hash]) -def get_one_line_commits(start_hash, end_hash): - return run_cmd(["git", "log", "--oneline", "%s..%s" % (start_hash, end_hash)]) -def num_commits_in_range(start_hash, end_hash): - output = run_cmd(["git", "log", "--oneline", "%s..%s" % (start_hash, end_hash)]) - lines = [line for line in output.split("\n") if line] # filter out empty lines - return len(lines) +def tag_exists(tag): + stderr = run_cmd_error(["git", "show", tag]) + return "error" not in stderr + +# A type-safe representation of a commit +class Commit: + def __init__(self, _hash, author, title, pr_number = None): + self._hash = _hash + self.author = author + self.title = title + self.pr_number = pr_number + def get_hash(self): return self._hash + def get_author(self): return self.author + def get_title(self): return self.title + def get_pr_number(self): return self.pr_number + def __str__(self): + closes_pr = "(Closes #%s)" % self.pr_number if self.pr_number else "" + return "%s %s %s %s" % (self._hash, self.author, self.title, closes_pr) + +# Return all commits that belong to the specified tag. +# +# Under the hood, this runs a `git log` on that tag and parses the fields +# from the command output to construct a list of Commit objects. Note that +# because certain fields reside in the commit description and cannot be parsed +# through the Github API itself, we need to do some intelligent regex parsing +# to extract those fields. +# +# This is written using Git 1.8.5. +def get_commits(tag): + commit_start_marker = "|=== COMMIT START MARKER ===|" + commit_end_marker = "|=== COMMIT END MARKER ===|" + field_end_marker = "|=== COMMIT FIELD END MARKER ===|" + log_format =\ + commit_start_marker + "%h" +\ + field_end_marker + "%an" +\ + field_end_marker + "%s" +\ + commit_end_marker + "%b" + output = run_cmd(["git", "log", "--quiet", "--pretty=format:" + log_format, tag]) + commits = [] + raw_commits = [c for c in output.split(commit_start_marker) if c] + for commit in raw_commits: + if commit.count(commit_end_marker) != 1: + print "Commit end marker not found in commit: " + for line in commit.split("\n"): print line + sys.exit(1) + # Separate commit digest from the body + # From the digest we extract the hash, author and the title + # From the body, we extract the PR number and the github username + [commit_digest, commit_body] = commit.split(commit_end_marker) + if commit_digest.count(field_end_marker) != 2: + sys.exit("Unexpected format in commit: %s" % commit_digest) + [_hash, author, title] = commit_digest.split(field_end_marker) + # The PR number and github username is in the commit message + # itself and cannot be accessed through any Github API + pr_number = None + match = re.search("Closes #([0-9]+) from ([^/\\s]+)/", commit_body) + if match: + [pr_number, github_username] = match.groups() + # If the author name is not valid, use the github + # username so we can translate it properly later + if not is_valid_author(author): + author = github_username + # Guard against special characters + author = unidecode.unidecode(unicode(author, "UTF-8")).strip() + commit = Commit(_hash, author, title, pr_number) + commits.append(commit) + return commits # Maintain a mapping for translating issue types to contributions in the release notes # This serves an additional function of warning the user against unknown issue types @@ -70,10 +136,13 @@ def num_commits_in_range(start_hash, end_hash): known_issue_types = { "bug": "bug fixes", "build": "build fixes", + "dependency upgrade": "build fixes", "improvement": "improvements", "new feature": "new features", "documentation": "documentation", - "test": "test" + "test": "test", + "task": "improvement", + "sub-task": "improvement" } # Maintain a mapping for translating component names when creating the release notes @@ -176,8 +245,7 @@ def get_jira_name(author, jira_client): # Return whether the given name is in the form def is_valid_author(author): if not author: return False - author_words = len(author.split(" ")) - return author_words == 2 or author_words == 3 + return " " in author and not re.findall("[0-9]", author) # Capitalize the first letter of each word in the given author name def capitalize_author(author): diff --git a/dev/create-release/translate-contributors.py b/dev/create-release/translate-contributors.py index ef4625b003cb6..86fa02d87b9a0 100755 --- a/dev/create-release/translate-contributors.py +++ b/dev/create-release/translate-contributors.py @@ -37,17 +37,18 @@ JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") JIRA_USERNAME = os.environ.get("JIRA_USERNAME", None) JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", None) +GITHUB_API_TOKEN = os.environ.get("GITHUB_API_TOKEN", None) if not JIRA_USERNAME or not JIRA_PASSWORD: sys.exit("Both JIRA_USERNAME and JIRA_PASSWORD must be set") +if not GITHUB_API_TOKEN: + sys.exit("GITHUB_API_TOKEN must be set") -# Write new contributors list to .new +# Write new contributors list to .final if not os.path.isfile(contributors_file_name): print "Contributors file %s does not exist!" % contributors_file_name print "Have you run ./generate-contributors.py yet?" sys.exit(1) contributors_file = open(contributors_file_name, "r") -new_contributors_file_name = contributors_file_name + ".new" -new_contributors_file = open(new_contributors_file_name, "w") warnings = [] # In non-interactive mode, this script will choose the first replacement that is valid @@ -62,7 +63,20 @@ # Setup Github and JIRA clients jira_options = { "server": JIRA_API_BASE } jira_client = JIRA(options = jira_options, basic_auth = (JIRA_USERNAME, JIRA_PASSWORD)) -github_client = Github() +github_client = Github(GITHUB_API_TOKEN) + +# Load known author translations that are cached locally +known_translations = {} +known_translations_file_name = "known_translations" +known_translations_file = open(known_translations_file_name, "r") +for line in known_translations_file: + if line.startswith("#"): continue + [old_name, new_name] = line.strip("\n").split(" - ") + known_translations[old_name] = new_name +known_translations_file.close() + +# Open again in case the user adds new mappings +known_translations_file = open(known_translations_file_name, "a") # Generate candidates for the given author. This should only be called if the given author # name does not represent a full name as this operation is somewhat expensive. Under the @@ -80,21 +94,28 @@ def generate_candidates(author, issues): candidates = [] # First check for full name of Github user - github_name = get_github_name(new_author, github_client) + github_name = get_github_name(author, github_client) if github_name: - candidates.append((github_name, "Full name of Github user %s" % new_author)) + candidates.append((github_name, "Full name of Github user %s" % author)) else: - candidates.append((NOT_FOUND, "No full name found for Github user %s" % new_author)) + candidates.append((NOT_FOUND, "No full name found for Github user %s" % author)) # Then do the same for JIRA user - jira_name = get_jira_name(new_author, jira_client) + jira_name = get_jira_name(author, jira_client) if jira_name: - candidates.append((jira_name, "Full name of JIRA user %s" % new_author)) + candidates.append((jira_name, "Full name of JIRA user %s" % author)) else: - candidates.append((NOT_FOUND, "No full name found for JIRA user %s" % new_author)) + candidates.append((NOT_FOUND, "No full name found for JIRA user %s" % author)) # Then do the same for the assignee of each of the associated JIRAs # Note that a given issue may not have an assignee, or the assignee may not have a full name for issue in issues: - jira_issue = jira_client.issue(issue) + try: + jira_issue = jira_client.issue(issue) + except JIRAError as e: + # Do not exit just because an issue is not found! + if e.status_code == 404: + warnings.append("Issue %s not found!" % issue) + continue + raise e jira_assignee = jira_issue.fields.assignee if jira_assignee: user_name = jira_assignee.name @@ -123,42 +144,61 @@ def generate_candidates(author, issues): # In non-interactive mode, this script picks the first valid author name from the candidates # If no such name exists, the original name is used (without the JIRA numbers). print "\n========================== Translating contributor list ==========================" -for line in contributors_file: - author = line.split(" - ")[0] - print "Processing author %s" % author - if not author: - print " ERROR: Expected the following format - " - print " ERROR: Actual = %s" % line - if not is_valid_author(author): - new_author = author.split("/")[0] - issues = author.split("/")[1:] - candidates = generate_candidates(new_author, issues) +lines = contributors_file.readlines() +contributions = [] +for i, line in enumerate(lines): + temp_author = line.strip(" * ").split(" -- ")[0] + print "Processing author %s (%d/%d)" % (temp_author, i + 1, len(lines)) + if not temp_author: + error_msg = " ERROR: Expected the following format \" * -- \"\n" + error_msg += " ERROR: Actual = %s" % line + print error_msg + warnings.append(error_msg) + contributions.append(line) + continue + author = temp_author.split("/")[0] + # Use the local copy of known translations where possible + if author in known_translations: + line = line.replace(temp_author, known_translations[author]) + elif not is_valid_author(author): + new_author = author + issues = temp_author.split("/")[1:] + candidates = generate_candidates(author, issues) # Print out potential replacement candidates along with the sources, e.g. # [X] No full name found for Github user andrewor14 + # [X] No assignee found for SPARK-1763 # [0] Andrew Or - Full name of JIRA user andrewor14 # [1] Andrew Orso - Full name of SPARK-1444 assignee andrewor14 # [2] Andrew Ordall - Full name of SPARK-1663 assignee andrewor14 - # [X] No assignee found for SPARK-1763 - # [3] Custom + # [3] andrewor14 - Raw Github username + # [4] Custom candidate_names = [] + bad_prompts = [] # Prompts that can't actually be selected; print these first. + good_prompts = [] # Prompts that contain valid choices for candidate, source in candidates: if candidate == NOT_FOUND: - print " [X] %s" % source + bad_prompts.append(" [X] %s" % source) else: index = len(candidate_names) candidate_names.append(candidate) - print " [%d] %s - %s" % (index, candidate, source) - custom_index = len(candidate_names) + good_prompts.append(" [%d] %s - %s" % (index, candidate, source)) + raw_index = len(candidate_names) + custom_index = len(candidate_names) + 1 + for p in bad_prompts: print p + if bad_prompts: print " ---" + for p in good_prompts: print p # In interactive mode, additionally provide "custom" option and await user response if INTERACTIVE_MODE: + print " [%d] %s - Raw Github username" % (raw_index, author) print " [%d] Custom" % custom_index response = raw_input(" Your choice: ") - while not response.isdigit() or int(response) > custom_index: - response = raw_input(" Please enter an integer between 0 and %d: " % custom_index) + last_index = custom_index + while not response.isdigit() or int(response) > last_index: + response = raw_input(" Please enter an integer between 0 and %d: " % last_index) response = int(response) if response == custom_index: new_author = raw_input(" Please type a custom name for this author: ") - else: + elif response != raw_index: new_author = candidate_names[response] # In non-interactive mode, just pick the first candidate else: @@ -171,12 +211,35 @@ def generate_candidates(author, issues): if is_valid_author(new_author): new_author = capitalize_author(new_author) else: - warnings.append("Unable to find a valid name %s for author %s" % (new_author, author)) + warnings.append("Unable to find a valid name %s for author %s" % (author, temp_author)) print " * Replacing %s with %s" % (author, new_author) - line = line.replace(author, new_author) - new_contributors_file.write(line) + # If we are in interactive mode, prompt the user whether we want to remember this new mapping + if INTERACTIVE_MODE and\ + author not in known_translations and\ + yesOrNoPrompt(" Add mapping %s -> %s to known translations file?" % (author, new_author)): + known_translations_file.write("%s - %s\n" % (author, new_author)) + known_translations_file.flush() + line = line.replace(temp_author, author) + contributions.append(line) print "==================================================================================\n" contributors_file.close() +known_translations_file.close() + +# Sort the contributions before writing them to the new file. +# Additionally, check if there are any duplicate author rows. +# This could happen if the same user has both a valid full +# name (e.g. Andrew Or) and an invalid one (andrewor14). +# If so, warn the user about this at the end. +contributions.sort() +all_authors = set() +new_contributors_file_name = contributors_file_name + ".final" +new_contributors_file = open(new_contributors_file_name, "w") +for line in contributions: + author = line.strip(" * ").split(" -- ")[0] + if author in all_authors: + warnings.append("Detected duplicate author name %s. Please merge these manually." % author) + all_authors.add(author) + new_contributors_file.write(line) new_contributors_file.close() print "Translated contributors list successfully written to %s!" % new_contributors_file_name diff --git a/dev/mima b/dev/mima index 40603166c21ae..bed5cd042634e 100755 --- a/dev/mima +++ b/dev/mima @@ -24,13 +24,13 @@ set -e FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" -echo -e "q\n" | sbt/sbt oldDeps/update +echo -e "q\n" | build/sbt oldDeps/update rm -f .generated-mima* -# Generate Mima Ignore is called twice, first with latest built jars +# Generate Mima Ignore is called twice, first with latest built jars # on the classpath and then again with previous version jars on the classpath. # Because of a bug in GenerateMIMAIgnore that when old jars are ahead on classpath -# it did not process the new classes (which are in assembly jar). +# it did not process the new classes (which are in assembly jar). ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore export SPARK_CLASSPATH="`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"`" @@ -38,7 +38,7 @@ echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore -echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" +echo -e "q\n" | build/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" ret_val=$? if [ $ret_val != 0 ]; then diff --git a/dev/run-tests b/dev/run-tests index 328a73bd8b26d..20603fc089239 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -59,17 +59,17 @@ export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" { if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" - else + else declare java_cmd=java fi - + # We can't use sed -r -e due to OS X / BSD compatibility; hence, all the parentheses. JAVA_VERSION=$( $java_cmd -version 2>&1 \ | grep -e "^java version" --max-count=1 \ | sed "s/java version \"\(.*\)\.\(.*\)\.\(.*\)\"/\1\2/" ) - + if [ "$JAVA_VERSION" -lt 18 ]; then echo "[warn] Java 8 tests will not run because JDK version is < 1.8." fi @@ -79,7 +79,7 @@ export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" # Partial solution for SPARK-1455. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master - + sql_diffs=$( git diff --name-only master \ | grep -e "^sql/" -e "^bin/spark-sql" -e "^sbin/start-thriftserver.sh" @@ -93,7 +93,7 @@ if [ -n "$AMPLAB_JENKINS" ]; then if [ -n "$sql_diffs" ]; then echo "[info] Detected changes in SQL. Will run Hive test suite." _RUN_SQL_TESTS=true - + if [ -z "$non_sql_diffs" ]; then echo "[info] Detected no changes except in SQL. Will only run SQL tests." _SQL_TESTS_ONLY=true @@ -141,24 +141,26 @@ CURRENT_BLOCK=$BLOCK_BUILD { # NOTE: echo "q" is needed because sbt on encountering a build file with failure - #+ (either resolution or compilation) prompts the user for input either q, r, etc - #+ to quit or retry. This echo is there to make it not block. + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a - #+ single argument! + # single argument! # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? - # First build with 0.12 to ensure patches do not break the hive 12 build + # First build with Hive 0.12.0 to ensure patches do not break the Hive 0.12.0 build HIVE_12_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver -Phive-0.12.0" - echo "[info] Compile with hive 0.12" + echo "[info] Compile with Hive 0.12.0" echo -e "q\n" \ - | sbt/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ + | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" - # Then build with default version(0.13.1) because tests are based on this version + # Then build with default Hive version (0.13.1) because tests are based on this version + echo "[info] Compile with Hive 0.13.1" + rm -rf lib_managed echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS"\ " -Phive -Phive-thriftserver" echo -e "q\n" \ - | sbt/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \ + | build/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } @@ -175,27 +177,27 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_RUN_SQL_TESTS" ]; then SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" fi - + if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string - #+ will be interpreted as a single test, which doesn't work. + # will be interpreted as a single test, which doesn't work. SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "mllib/test") else SBT_MAVEN_TEST_ARGS=("test") fi - + echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" - + # NOTE: echo "q" is needed because sbt on encountering a build file with failure - #+ (either resolution or compilation) prompts the user for input either q, r, etc - #+ to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a - #+ single argument! - #+ "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a + # single argument! + # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ - | sbt/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ + | build/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } @@ -210,7 +212,7 @@ CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS echo "" echo "=========================================================================" -echo "Detecting binary incompatibilites with MiMa" +echo "Detecting binary incompatibilities with MiMa" echo "=========================================================================" CURRENT_BLOCK=$BLOCK_MIMA diff --git a/dev/scalastyle b/dev/scalastyle index 3a4df6e4bf1bc..86919227ed1ab 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,9 +17,9 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | build/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt # Check style with YARN built too -echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ +echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ >> scalastyle.txt ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') diff --git a/docs/README.md b/docs/README.md index 119484038083f..8a54724c4beae 100644 --- a/docs/README.md +++ b/docs/README.md @@ -21,7 +21,7 @@ read those text files directly if you want. Start with index.md. The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllrb.com). `Jekyll` and a few dependencies must be installed for this to work. We recommend -installing via the Ruby Gem dependency manager. Since the exact HTML output +installing via the Ruby Gem dependency manager. Since the exact HTML output varies between versions of Jekyll and its dependencies, we list specific versions here in some cases: @@ -60,7 +60,7 @@ We use Sphinx to generate Python API docs, so you will need to install it by run ## API Docs (Scaladoc and Sphinx) -You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. +You can build just the Spark scaladoc by running `build/sbt doc` from the SPARK_PROJECT_ROOT directory. Similarly, you can build just the PySpark docs by running `make html` from the SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for classes that are listed as @@ -68,7 +68,7 @@ public in `__init__.py`. When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a -jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it +jekyll plugin to run `build/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs [Sphinx](http://sphinx-doc.org/). diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 4566a2fff562b..3c626a0b7f54b 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -25,8 +25,8 @@ curr_dir = pwd cd("..") - puts "Running 'sbt/sbt -Pkinesis-asl compile unidoc' from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt -Pkinesis-asl compile unidoc` + puts "Running 'build/sbt -Pkinesis-asl compile unidoc' from " + pwd + "; this may take a few minutes..." + puts `build/sbt -Pkinesis-asl compile unidoc` puts "Moving back into docs dir." cd("docs") diff --git a/docs/building-spark.md b/docs/building-spark.md index 4922e877e9b78..fb93017861ed0 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -9,6 +9,15 @@ redirect_from: "building-with-maven.html" Building Spark using Maven requires Maven 3.0.4 or newer and Java 6+. +# Building with `build/mvn` + +Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: + +{% highlight bash %} +build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package +{% endhighlight %} + +Other build examples can be found below. # Setting up Maven's Memory Usage @@ -28,7 +37,9 @@ If you don't run this, you may see errors like the following: You can fix this by setting the `MAVEN_OPTS` variable as discussed before. -**Note:** *For Java 8 and above this step is not required.* +**Note:** +* *For Java 8 and above this step is not required.* +* *If using `build/mvn` and `MAVEN_OPTS` were not already set, the script will automate this for you.* # Specifying the Hadoop Version @@ -60,7 +71,7 @@ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package {% endhighlight %} -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN, you can enable the "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". As of Spark 1.3, Spark only supports YARN versions 2.2.0 and later. +You can enable the "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". Spark only supports YARN versions 2.2.0 and later. Examples: @@ -84,7 +95,7 @@ mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests # Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. -By default Spark will build with Hive 0.13.1 bindings. You can also build for +By default Spark will build with Hive 0.13.1 bindings. You can also build for Hive 0.12.0 using the `-Phive-0.12.0` profile. {% highlight bash %} # Apache Hadoop 2.4.X with Hive 13 support @@ -106,7 +117,7 @@ supported in Scala 2.11 builds. # Spark Tests in Maven -Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: @@ -124,11 +135,26 @@ We use the scala-maven-plugin which supports incremental and continuous compilat mvn scala:cc -should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. +should run continuous compilation (i.e. wait for changes). However, this has not been tested +extensively. A couple of gotchas to note: +* it only scans the paths `src/main` and `src/test` (see +[docs](http://scala-tools.org/mvnsites/maven-scala-plugin/usage_cc.html)), so it will only work +from within certain submodules that have that structure. +* you'll typically need to run `mvn install` from the project root for compilation within +specific submodules to work; this is because submodules that depend on other submodules do so via +the `spark-parent` module). + +Thus, the full flow for running continuous-compilation of the `core` submodule may look more like: + ``` + $ mvn install + $ cd core + $ mvn scala:cc +``` -# Using With IntelliJ IDEA +# Building Spark with IntelliJ IDEA or Eclipse -This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the pom.xml file in the project root folder, you only need to activate either the hadoop1 or hadoop2 profile in the "Maven Properties" popout. We have not tried Eclipse/Scala IDE with this. +For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the +[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-IDESetup). # Building Spark Debian Packages @@ -143,9 +169,9 @@ The debian package can then be found under assembly/target. We added the short c Running only Java 8 tests and nothing else. mvn install -DskipTests -Pjava8-tests - -Java 8 tests are run when `-Pjava8-tests` profile is enabled, they will run in spite of `-DskipTests`. -For these tests to run your system must have a JDK 8 installation. + +Java 8 tests are run when `-Pjava8-tests` profile is enabled, they will run in spite of `-DskipTests`. +For these tests to run your system must have a JDK 8 installation. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. # Building for PySpark on YARN @@ -157,7 +183,7 @@ then ship it over to the cluster. We are investigating the exact cause for this. # Packaging without Hadoop Dependencies for YARN -The assembly jar produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. +The assembly jar produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. # Building with SBT @@ -168,22 +194,22 @@ compilation. More advanced developers may wish to use SBT. The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables can be set to control the SBT build. For example: - sbt/sbt -Pyarn -Phadoop-2.3 assembly + build/sbt -Pyarn -Phadoop-2.3 assembly # Testing with SBT -Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: +Some of the tests require Spark to be packaged first, so always run `build/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver assembly - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver assembly + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test To run only a specific test suite as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" To run test suites of a specific sub project as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test # Speeding up Compilation with Zinc @@ -192,3 +218,9 @@ compiler. When run locally as a background process, it speeds up builds of Scala like Spark. Developers who regularly recompile Spark with Maven will be the most interested in Zinc. The project site gives instructions for building and running `zinc`; OS X users can install it using `brew install zinc`. + +If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all +builds. This process will auto-start after the first time `build/mvn` is called and bind to port +3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be +shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically +restart whenever `build/mvn` is called. diff --git a/docs/configuration.md b/docs/configuration.md index acee267883ed5..2add48569bece 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -75,8 +75,8 @@ in the `spark-defaults.conf` file. The application web UI at `http://:4040` lists Spark properties in the "Environment" tab. This is a useful place to check to make sure that your properties have been set correctly. Note -that only values explicitly specified through either `spark-defaults.conf` or SparkConf will -appear. For all other configuration properties, you can assume the default value is used. +that only values explicitly specified through `spark-defaults.conf`, `SparkConf`, or the command +line will appear. For all other configuration properties, you can assume the default value is used. ## Available Properties @@ -310,7 +310,7 @@ Apart from these, the following properties are also available, and may be useful (none) Add the environment variable specified by EnvironmentVariableName to the Executor - process. The user can specify multiple of these and to set multiple environment variables. + process. The user can specify multiple of these to set multiple environment variables. @@ -452,7 +452,7 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedJobs 1000 - How many stages the Spark UI and status APIs remember before garbage + How many jobs the Spark UI and status APIs remember before garbage collecting. @@ -709,7 +709,9 @@ Apart from these, the following properties are also available, and may be useful If set to true, validates the output specification (e.g. checking if the output directory already exists) used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing output directories. We recommend that users do not disable this except if trying to achieve compatibility with - previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since + data may need to be rewritten to pre-existing output directories during checkpoint recovery. spark.hadoop.cloneConf @@ -816,6 +818,16 @@ Apart from these, the following properties are also available, and may be useful Communication timeout between Spark nodes, in seconds. + + spark.network.timeout + 120 + + Default timeout for all network interactions, in seconds. This config will be used in + place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, + spark.storage.blockManagerSlaveTimeoutMs or + spark.shuffle.io.connectionTimeout, if they are not configured. + + spark.akka.heartbeat.pauses 6000 @@ -852,6 +864,41 @@ Apart from these, the following properties are also available, and may be useful between nodes leading to flooding the network with those. + + spark.shuffle.io.preferDirectBufs + true + + (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache + block transfer. For environments where off-heap memory is tightly limited, users may wish to + turn this off to force all allocations from Netty to be on-heap. + + + + spark.shuffle.io.numConnectionsPerPeer + 1 + + (Netty only) Connections between hosts are reused in order to reduce connection buildup for + large clusters. For clusters with many hard disks and few hosts, this may result in insufficient + concurrency to saturate all disks, and so users may consider increasing this value. + + + + spark.shuffle.io.maxRetries + 3 + + (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is + set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC + pauses or transient network connectivity issues. + + + + spark.shuffle.io.retryWait + 5 + + (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying + is simply maxRetries * retryWait, by default 15 seconds. + + #### Scheduling @@ -1008,6 +1055,67 @@ Apart from these, the following properties are also available, and may be useful +#### Dynamic allocation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.dynamicAllocation.enabledfalse + Whether to use dynamic resource allocation, which scales the number of executors registered + with this application up and down based on the workload. Note that this is currently only + available on YARN mode. For more detail, see the description + here. +

+ This requires the following configurations to be set: + spark.dynamicAllocation.minExecutors, + spark.dynamicAllocation.maxExecutors, and + spark.shuffle.service.enabled +
spark.dynamicAllocation.minExecutors(none) + Lower bound for the number of executors if dynamic allocation is enabled (required). +
spark.dynamicAllocation.maxExecutors(none) + Upper bound for the number of executors if dynamic allocation is enabled (required). +
spark.dynamicAllocation.schedulerBacklogTimeout60 + If dynamic allocation is enabled and there have been pending tasks backlogged for more than + this duration (in seconds), new executors will be requested. For more detail, see this + description. +
spark.dynamicAllocation.sustainedSchedulerBacklogTimeoutschedulerBacklogTimeout + Same as spark.dynamicAllocation.schedulerBacklogTimeout, but used only for + subsequent executor requests. For more detail, see this + description. +
spark.dynamicAllocation.executorIdleTimeout600 + If dynamic allocation is enabled and an executor has been idle for more than this duration + (in seconds), the executor will be removed. For more detail, see this + description. +
+ #### Security diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index ed51d0abb3a45..d50f445d7ecc7 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -94,6 +94,25 @@ another. permissions on your private key file, you can run `launch` with the `--resume` option to restart the setup process on an existing cluster. +# Launching a Cluster in a VPC + +- Run + `./spark-ec2 -k -i -s --vpc-id= --subnet-id= launch `, + where `` is the name of your EC2 key pair (that you gave it + when you created it), `` is the private key file for your + key pair, `` is the number of slave nodes to launch (try + 1 at first), `` is the name of your VPC, `` is the + name of your subnet, and `` is the name to give to your + cluster. + + For example: + + ```bash + export AWS_SECRET_ACCESS_KEY=AaBbCcDdEeFGgHhIiJjKkLlMmNnOoPpQqRrSsTtU +export AWS_ACCESS_KEY_ID=ABCDEFG1234567890123 +./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a --vpc-id=vpc-a28d24c7 --subnet-id=subnet-4eb27b39 --spark-version=1.1.0 launch my-spark-cluster + ``` + # Running Applications - Go into the `ec2` directory in the release of Spark you downloaded. diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index dd73e9dc54440..87dcc58feb494 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -18,7 +18,7 @@ see the guide on [building with maven](building-spark.html#specifying-the-hadoop The table below lists the corresponding `hadoop.version` code for each CDH/HDP release. Note that some Hadoop releases are binary compatible across client versions. This means the pre-built Spark -distribution may "just work" without you needing to compile. That said, we recommend compiling with +distribution may "just work" without you needing to compile. That said, we recommend compiling with the _exact_ Hadoop version you are running to avoid any compatibility errors.
Property NameDefaultMeaning
@@ -50,7 +50,7 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors. In SBT, the equivalent can be achieved by setting the the `hadoop.version` property: - sbt/sbt -Dhadoop.version=1.0.4 assembly + build/sbt -Dhadoop.version=1.0.4 assembly # Linking Applications to the Hadoop Version @@ -98,11 +98,11 @@ Spark can run in a variety of deployment modes: * Using dedicated set of Spark nodes in your cluster. These nodes should be co-located with your Hadoop installation. -* Running on the same nodes as an existing Hadoop installation, with a fixed amount memory and +* Running on the same nodes as an existing Hadoop installation, with a fixed amount memory and cores dedicated to Spark on each node. * Run Spark alongside Hadoop using a cluster resource manager, such as YARN or Mesos. -These options are identical for those using CDH and HDP. +These options are identical for those using CDH and HDP. # Inheriting Cluster Configuration @@ -116,5 +116,5 @@ The location of these configuration files varies across CDH and HDP versions, bu a common location is inside of `/etc/hadoop/conf`. Some tools, such as Cloudera Manager, create configurations on-the-fly, but offer a mechanisms to download copies of them. -To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` +To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` to a location containing the configuration files. diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 94604f301dd46..a5425eb3557b2 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -56,6 +56,114 @@ the same RDDs. For example, the [Shark](http://shark.cs.berkeley.edu) JDBC serve queries. In future releases, in-memory storage systems such as [Tachyon](http://tachyon-project.org) will provide another approach to share RDDs. +## Dynamic Resource Allocation + +Spark 1.2 introduces the ability to dynamically scale the set of cluster resources allocated to +your application up and down based on the workload. This means that your application may give +resources back to the cluster if they are no longer used and request them again later when there +is demand. This feature is particularly useful if multiple applications share resources in your +Spark cluster. If a subset of the resources allocated to an application becomes idle, it can be +returned to the cluster's pool of resources and acquired by other applications. In Spark, dynamic +resource allocation is performed on the granularity of the executor and can be enabled through +`spark.dynamicAllocation.enabled`. + +This feature is currently disabled by default and available only on [YARN](running-on-yarn.html). +A future release will extend this to [standalone mode](spark-standalone.html) and +[Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes). Note that although Spark on +Mesos already has a similar notion of dynamic resource sharing in fine-grained mode, enabling +dynamic allocation allows your Mesos application to take advantage of coarse-grained low-latency +scheduling while sharing cluster resources efficiently. + +### Configuration and Setup + +All configurations used by this feature live under the `spark.dynamicAllocation.*` namespace. +To enable this feature, your application must set `spark.dynamicAllocation.enabled` to `true` and +provide lower and upper bounds for the number of executors through +`spark.dynamicAllocation.minExecutors` and `spark.dynamicAllocation.maxExecutors`. Other relevant +configurations are described on the [configurations page](configuration.html#dynamic-allocation) +and in the subsequent sections in detail. + +Additionally, your application must use an external shuffle service. The purpose of the service is +to preserve the shuffle files written by executors so the executors can be safely removed (more +detail described [below](job-scheduling.html#graceful-decommission-of-executors)). To enable +this service, set `spark.shuffle.service.enabled` to `true`. In YARN, this external shuffle service +is implemented in `org.apache.spark.yarn.network.YarnShuffleService` that runs in each `NodeManager` +in your cluster. To start this service, follow these steps: + +1. Build Spark with the [YARN profile](building-spark.html). Skip this step if you are using a +pre-packaged distribution. +2. Locate the `spark--yarn-shuffle.jar`. This should be under +`$SPARK_HOME/network/yarn/target/scala-` if you are building Spark yourself, and under +`lib` if you are using a distribution. +2. Add this jar to the classpath of all `NodeManager`s in your cluster. +3. In the `yarn-site.xml` on each node, add `spark_shuffle` to `yarn.nodemanager.aux-services`, +then set `yarn.nodemanager.aux-services.spark_shuffle.class` to +`org.apache.spark.network.yarn.YarnShuffleService`. Additionally, set all relevant +`spark.shuffle.service.*` [configurations](configuration.html). +4. Restart all `NodeManager`s in your cluster. + +### Resource Allocation Policy + +At a high level, Spark should relinquish executors when they are no longer used and acquire +executors when they are needed. Since there is no definitive way to predict whether an executor +that is about to be removed will run a task in the near future, or whether a new executor that is +about to be added will actually be idle, we need a set of heuristics to determine when to remove +and request executors. + +#### Request Policy + +A Spark application with dynamic allocation enabled requests additional executors when it has +pending tasks waiting to be scheduled. This condition necessarily implies that the existing set +of executors is insufficient to simultaneously saturate all tasks that have been submitted but +not yet finished. + +Spark requests executors in rounds. The actual request is triggered when there have been pending +tasks for `spark.dynamicAllocation.schedulerBacklogTimeout` seconds, and then triggered again +every `spark.dynamicAllocation.sustainedSchedulerBacklogTimeout` seconds thereafter if the queue +of pending tasks persists. Additionally, the number of executors requested in each round increases +exponentially from the previous round. For instance, an application will add 1 executor in the +first round, and then 2, 4, 8 and so on executors in the subsequent rounds. + +The motivation for an exponential increase policy is twofold. First, an application should request +executors cautiously in the beginning in case it turns out that only a few additional executors is +sufficient. This echoes the justification for TCP slow start. Second, the application should be +able to ramp up its resource usage in a timely manner in case it turns out that many executors are +actually needed. + +#### Remove Policy + +The policy for removing executors is much simpler. A Spark application removes an executor when +it has been idle for more than `spark.dynamicAllocation.executorIdleTimeout` seconds. Note that, +under most circumstances, this condition is mutually exclusive with the request condition, in that +an executor should not be idle if there are still pending tasks to be scheduled. + +### Graceful Decommission of Executors + +Before dynamic allocation, a Spark executor exits either on failure or when the associated +application has also exited. In both scenarios, all state associated with the executor is no +longer needed and can be safely discarded. With dynamic allocation, however, the application +is still running when an executor is explicitly removed. If the application attempts to access +state stored in or written by the executor, it will have to perform a recompute the state. Thus, +Spark needs a mechanism to decommission an executor gracefully by preserving its state before +removing it. + +This requirement is especially important for shuffles. During a shuffle, the Spark executor first +writes its own map outputs locally to disk, and then acts as the server for those files when other +executors attempt to fetch them. In the event of stragglers, which are tasks that run for much +longer than their peers, dynamic allocation may remove an executor before the shuffle completes, +in which case the shuffle files written by that executor must be recomputed unnecessarily. + +The solution for preserving shuffle files is to use an external shuffle service, also introduced +in Spark 1.2. This service refers to a long-running process that runs on each node of your cluster +independently of your Spark applications and their executors. If the service is enabled, Spark +executors will fetch shuffle files from the service instead of from each other. This means any +shuffle state written by an executor may continue to be served beyond the executor's lifetime. + +In addition to writing shuffle files, executors also cache data either on disk or in memory. +When an executor is removed, however, all cached data will no longer be accessible. There is +currently not yet a solution for this in Spark 1.2. In future releases, the cached data may be +preserved through an off-heap storage similar in spirit to how shuffle files are preserved through +the external shuffle service. # Scheduling Within an Application diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 1073abb202c56..78358499fd01f 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -2,6 +2,8 @@ layout: global title: Running Spark on Mesos --- +* This will become a table of contents (this text will be scraped). +{:toc} Spark can run on hardware clusters managed by [Apache Mesos](http://mesos.apache.org/). @@ -183,6 +185,49 @@ node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop). In either case, HDFS runs separately from Hadoop MapReduce, without being scheduled through Mesos. +# Configuration + +See the [configuration page](configuration.html) for information on Spark configurations. The following configs are specific for Spark on Mesos. + +#### Spark Properties + +
+ + + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.mesos.coarsefalse + Set the run mode for Spark on Mesos. For more information about the run mode, refer to #Mesos Run Mode section above. +
spark.mesos.extra.cores0 + Set the extra amount of cpus to request per task. This setting is only used for Mesos coarse grain mode. + The total amount of cores requested per task is the number of cores in the offer plus the extra cores configured. + Note that total amount of cores the executor will request in total will not exceed the spark.cores.max setting. +
spark.mesos.executor.homeSPARK_HOME + The location where the mesos executor will look for Spark binaries to execute, and uses the SPARK_HOME setting on default. + This variable is only used when no spark.executor.uri is provided, and assumes Spark is installed on the specified location + on each slave. +
spark.mesos.executor.memoryOverhead384 + The amount of memory that Mesos executor will request for the task to account for the overhead of running the executor itself. + The final total amount of memory allocated is the maximum value between executor memory plus memoryOverhead, and overhead fraction (1.07) plus the executor memory. +
+ # Troubleshooting and Debugging A few places to look during debugging: diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 62b317129b72a..183698ffe9304 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -22,10 +22,12 @@ Most of the configs are the same for Spark on YARN as for other deployment modes - - + + @@ -139,6 +141,22 @@ Most of the configs are the same for Spark on YARN as for other deployment modes The maximum number of threads to use in the application master for launching executor containers. + + + + + + + + + +
Property NameDefaultMeaning
spark.yarn.applicationMaster.waitTries10spark.yarn.am.waitTime100000 - Set the number of times the ApplicationMaster waits for the the Spark master and then also the number of tries it waits for the SparkContext to be initialized + In yarn-cluster mode, time in milliseconds for the application master to wait for the + SparkContext to be initialized. In yarn-client mode, time for the application master to wait + for the driver to connect to it.
spark.yarn.am.extraJavaOptions(none) + A string of extra JVM options to pass to the Yarn ApplicationMaster in client mode. + In cluster mode, use spark.driver.extraJavaOptions instead. +
spark.yarn.maxAppAttemptsyarn.resourcemanager.am.max-attempts in YARN + The maximum number of attempts that will be made to submit the application. + It should be no larger than the global number of max attempts in the YARN configuration. +
# Launching Spark on YARN @@ -201,18 +219,18 @@ settings and a restart of all node managers. Thus, this is not applicable to hos To use a custom log4j configuration for the application master or executors, there are two options: -- upload a custom log4j.properties using spark-submit, by adding it to the "--files" list of files +- upload a custom `log4j.properties` using `spark-submit`, by adding it to the `--files` list of files to be uploaded with the application. -- add "-Dlog4j.configuration=" to "spark.driver.extraJavaOptions" - (for the driver) or "spark.executor.extraJavaOptions" (for executors). Note that if using a file, - the "file:" protocol should be explicitly provided, and the file needs to exist locally on all +- add `-Dlog4j.configuration=` to `spark.driver.extraJavaOptions` + (for the driver) or `spark.executor.extraJavaOptions` (for executors). Note that if using a file, + the `file:` protocol should be explicitly provided, and the file needs to exist locally on all the nodes. Note that for the first option, both executors and the application master will share the same log4j configuration, which may cause issues when they run on the same node (e.g. trying to write to the same log file). -If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use "${spark.yarn.app.container.log.dir}" in your log4j.properties. For example, log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log. For streaming application, configuring RollingFileAppender and setting file location to YARN's log directory will avoid disk overflow caused by large log file, and logs can be accessed using YARN's log utility. +If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use `spark.yarn.app.container.log.dir` in your log4j.properties. For example, `log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log`. For streaming application, configuring `RollingFileAppender` and setting file location to YARN's log directory will avoid disk overflow caused by large log file, and logs can be accessed using YARN's log utility. # Important notes diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index be284fbe217a5..729045b81a8c0 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -625,6 +625,10 @@ This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +Note that the file that is offered as _jsonFile_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. As a consequence, +a regular multi-line JSON file will most often fail. + {% highlight scala %} // sc is an existing SparkContext. val sqlContext = new org.apache.spark.sql.SQLContext(sc) @@ -663,6 +667,10 @@ This conversion can be done using one of two methods in a JavaSQLContext : * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +Note that the file that is offered as _jsonFile_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. As a consequence, +a regular multi-line JSON file will most often fail. + {% highlight java %} // sc is an existing JavaSparkContext. JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); @@ -701,6 +709,10 @@ This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +Note that the file that is offered as _jsonFile_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. As a consequence, +a regular multi-line JSON file will most often fail. + {% highlight python %} # sc is an existing SparkContext. from pyspark.sql import SQLContext @@ -819,13 +831,10 @@ turning on some experimental options. ## Caching Data In Memory -Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")`. +Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")` or `schemaRDD.cache()`. Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. -Note that if you call `schemaRDD.cache()` rather than `sqlContext.cacheTable(...)`, tables will _not_ be cached using -the in-memory columnar format, and therefore `sqlContext.cacheTable(...)` is strongly recommended for this use case. - Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -938,6 +947,18 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may also use the beeline script that comes with Hive. +Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. +Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: + + hive.server2.transport.mode - Set this to value: http + hive.server2.thrift.http.port - HTTP port number fo listen on; default is 10001 + hive.server2.http.endpoint - HTTP endpoint; default is cliservice + +To test, use beeline to connect to the JDBC/ODBC server in http mode with: + + beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path= + + ## Running the Spark SQL CLI The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute @@ -986,12 +1007,11 @@ let user control table caching explicitly: CACHE TABLE logs_last_month; UNCACHE TABLE logs_last_month; -**NOTE:** `CACHE TABLE tbl` is lazy, similar to `.cache` on an RDD. This command only marks `tbl` to ensure that -partitions are cached when calculated but doesn't actually cache it until a query that touches `tbl` is executed. -To force the table to be cached, you may simply count the table immediately after executing `CACHE TABLE`: +**NOTE:** `CACHE TABLE tbl` is now __eager__ by default not __lazy__. Don’t need to trigger cache materialization manually anymore. - CACHE TABLE logs_last_month; - SELECT COUNT(1) FROM logs_last_month; +Spark SQL newly introduced a statement to let user control table caching whether or not lazy since Spark 1.2.0: + + CACHE [LAZY] TABLE [AS SELECT] ... Several caching related features are not supported yet: diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 1c956fcb40da8..0e38fe2144e9f 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -4,7 +4,7 @@ title: Spark Streaming + Kafka Integration Guide --- [Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. -1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). +1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} @@ -20,7 +20,7 @@ title: Spark Streaming + Kafka Integration Guide streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]) See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala).
import org.apache.spark.streaming.kafka.*; diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 1ac5b9e863ad4..01450efe35e55 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -75,7 +75,7 @@ main entry point for all streaming functionality. We create a local StreamingCon {% highlight scala %} import org.apache.spark._ import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+ // Create a local StreamingContext with two working thread and batch interval of 1 second. // The master requires 2 cores to prevent from a starvation scenario. @@ -107,7 +107,7 @@ each line will be split into multiple words and the stream of words is represent `words` DStream. Next, we want to count these words. {% highlight scala %} -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+ // Count each word in each batch val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 2581c9f69fa37..3bd1deaccfafe 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -10,7 +10,7 @@ through a uniform interface so you don't have to configure your application spec # Bundling Your Application's Dependencies If your code depends on other projects, you will need to package them alongside your application in order to distribute the code to a Spark cluster. To do this, -to create an assembly jar (or "uber" jar) containing your code and its dependencies. Both +create an assembly jar (or "uber" jar) containing your code and its dependencies. Both [sbt](https://github.com/sbt/sbt-assembly) and [Maven](http://maven.apache.org/plugins/maven-shade-plugin/) have assembly plugins. When creating assembly jars, list Spark and Hadoop @@ -59,7 +59,7 @@ for applications that involve the REPL (e.g. Spark shell). Alternatively, if your application is submitted from a machine far from the worker machines (e.g. locally on your laptop), it is common to use `cluster` mode to minimize network latency between the drivers and the executors. Note that `cluster` mode is currently not supported for standalone -clusters, Mesos clusters, or python applications. +clusters, Mesos clusters, or Python applications. For Python applications, simply pass a `.py` file in the place of `` instead of a JAR, and add Python `.zip`, `.egg` or `.py` files to the search path with `--py-files`. @@ -174,7 +174,7 @@ This can use up a significant amount of space over time and will need to be clea is handled automatically, and with Spark standalone, automatic cleanup can be configured with the `spark.worker.cleanup.appDataTtl` property. -For python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries +For Python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries to executors. # More Information diff --git a/docs/tuning.md b/docs/tuning.md index c4ca766328c1e..efaac9d3d405f 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -51,7 +51,7 @@ To register your own custom classes with Kryo, use the `registerKryoClasses` met {% highlight scala %} val conf = new SparkConf().setMaster(...).setAppName(...) -conf.registerKryoClasses(Seq(classOf[MyClass1], classOf[MyClass2])) +conf.registerKryoClasses(Array(classOf[MyClass1], classOf[MyClass2])) val sc = new SparkContext(conf) {% endhighlight %} @@ -111,7 +111,7 @@ pointer-based data structures and wrapper objects. There are several ways to do 3. Consider using numeric IDs or enumeration objects instead of strings for keys. 4. If you have less than 32 GB of RAM, set the JVM flag `-XX:+UseCompressedOops` to make pointers be four bytes instead of eight. You can add these options in - [`spark-env.sh`](configuration.html#environment-variables-in-spark-envsh). + [`spark-env.sh`](configuration.html#environment-variables). ## Serialized RDD Storage @@ -154,7 +154,7 @@ By default, Spark uses 60% of the configured executor memory (`spark.executor.me cache RDDs. This means that 40% of memory is available for any objects created during task execution. In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of -memory, lowering this value will help reduce the memory consumption. To change this to say 50%, you can call +memory, lowering this value will help reduce the memory consumption. To change this to, say, 50%, you can call `conf.set("spark.storage.memoryFraction", "0.5")` on your SparkConf. Combined with the use of serialized caching, using a smaller cache should be sufficient to mitigate most of the garbage collection problems. In case you are interested in further tuning the Java GC, continue reading below. @@ -190,7 +190,7 @@ temporary objects created during task execution. Some steps which may be useful * As an example, if your task is reading data from HDFS, the amount of memory used by the task can be estimated using the size of the data block read from HDFS. Note that the size of a decompressed block is often 2 or 3 times the - size of the block. So if we wish to have 3 or 4 tasks worth of working space, and the HDFS block size is 64 MB, + size of the block. So if we wish to have 3 or 4 tasks' worth of working space, and the HDFS block size is 64 MB, we can estimate size of Eden to be `4*3*64MB`. * Monitor how the frequency and time taken by garbage collection changes with the new settings. @@ -219,7 +219,7 @@ working set of one of your tasks, such as one of the reduce tasks in `groupByKey Spark's shuffle operations (`sortByKey`, `groupByKey`, `reduceByKey`, `join`, etc) build a hash table within each task to perform the grouping, which can often be large. The simplest fix here is to *increase the level of parallelism*, so that each task's input set is smaller. Spark can efficiently -support tasks as short as 200 ms, because it reuses one worker JVMs across all tasks and it has +support tasks as short as 200 ms, because it reuses one executor JVM across many tasks and it has a low task launching cost, so you can safely increase the level of parallelism to more than the number of cores in your clusters. diff --git a/ec2/spark-ec2 b/ec2/spark-ec2 index 4aa908242eeaa..3abd3f396f605 100755 --- a/ec2/spark-ec2 +++ b/ec2/spark-ec2 @@ -22,5 +22,4 @@ #+ the underlying Python script. SPARK_EC2_DIR="$(dirname $0)" -PYTHONPATH="${SPARK_EC2_DIR}/third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" \ - python "${SPARK_EC2_DIR}/spark_ec2.py" "$@" +python -Wdefault "${SPARK_EC2_DIR}/spark_ec2.py" "$@" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 5f9e484212635..abab209a05ba0 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -21,6 +21,7 @@ from __future__ import with_statement +import hashlib import logging import os import pipes @@ -29,6 +30,7 @@ import string import subprocess import sys +import tarfile import tempfile import time import urllib2 @@ -36,18 +38,64 @@ from datetime import datetime from optparse import OptionParser from sys import stderr -import boto -from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType -from boto import ec2 -DEFAULT_SPARK_VERSION = "1.1.0" +VALID_SPARK_VERSIONS = set([ + "0.7.3", + "0.8.0", + "0.8.1", + "0.9.0", + "0.9.1", + "0.9.2", + "1.0.0", + "1.0.1", + "1.0.2", + "1.1.0", + "1.1.1", + "1.2.0", +]) + +DEFAULT_SPARK_VERSION = "1.2.0" +DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) +MESOS_SPARK_EC2_BRANCH = "branch-1.3" -MESOS_SPARK_EC2_BRANCH = "v4" # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/{b}/ami-list".format(b=MESOS_SPARK_EC2_BRANCH) +def setup_boto(): + # Download Boto if it's not already present in the SPARK_EC2_DIR/lib folder: + version = "boto-2.34.0" + md5 = "5556223d2d0cc4d06dd4829e671dcecd" + url = "https://pypi.python.org/packages/source/b/boto/%s.tar.gz" % version + lib_dir = os.path.join(SPARK_EC2_DIR, "lib") + if not os.path.exists(lib_dir): + os.mkdir(lib_dir) + boto_lib_dir = os.path.join(lib_dir, version) + if not os.path.isdir(boto_lib_dir): + tgz_file_path = os.path.join(lib_dir, "%s.tar.gz" % version) + print "Downloading Boto from PyPi" + download_stream = urllib2.urlopen(url) + with open(tgz_file_path, "wb") as tgz_file: + tgz_file.write(download_stream.read()) + with open(tgz_file_path) as tar: + if hashlib.md5(tar.read()).hexdigest() != md5: + print >> stderr, "ERROR: Got wrong md5sum for Boto" + sys.exit(1) + tar = tarfile.open(tgz_file_path) + tar.extractall(path=lib_dir) + tar.close() + os.remove(tgz_file_path) + print "Finished downloading Boto" + sys.path.insert(0, boto_lib_dir) + + +setup_boto() +import boto +from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType +from boto import ec2 + + class UsageError(Exception): pass @@ -94,8 +142,8 @@ def parse_args(): help="Version of Spark to use: 'X.Y.Z' or a specific git hash (default: %default)") parser.add_option( "--spark-git-repo", - default="https://github.com/apache/spark", - help="Github repo from which to checkout supplied commit hash") + default=DEFAULT_SPARK_GITHUB_REPO, + help="Github repo from which to checkout supplied commit hash (default: %default)") parser.add_option( "--hadoop-major-version", default="1", help="Major version of Hadoop (default: %default)") @@ -120,6 +168,10 @@ def parse_args(): "Only possible on EBS-backed AMIs. " + "EBS volumes are only attached if --ebs-vol-size > 0." + "Only support up to 8 EBS volumes.") + parser.add_option("--placement-group", type="string", default=None, + help="Which placement group to try and launch " + + "instances into. Assumes placement group is already " + + "created.") parser.add_option( "--swap", metavar="SWAP", type="int", default=1024, help="Swap space to set up per node, in MB (default: %default)") @@ -162,6 +214,10 @@ def parse_args(): parser.add_option( "--copy-aws-credentials", action="store_true", default=False, help="Add AWS credentials to hadoop configuration to allow Spark to access S3") + parser.add_option( + "--subnet-id", default=None, help="VPC subnet to launch instances in") + parser.add_option( + "--vpc-id", default=None, help="VPC to launch instances in") (opts, args) = parser.parse_args() if len(args) != 2: @@ -186,14 +242,34 @@ def parse_args(): # Get the EC2 security group of the given name, creating it if it doesn't exist -def get_or_make_group(conn, name): +def get_or_make_group(conn, name, vpc_id): groups = conn.get_all_security_groups() group = [g for g in groups if g.name == name] if len(group) > 0: return group[0] else: print "Creating security group " + name - return conn.create_security_group(name, "Spark EC2 group") + return conn.create_security_group(name, "Spark EC2 group", vpc_id) + + +def get_validate_spark_version(version, repo): + if "." in version: + version = version.replace("v", "") + if version not in VALID_SPARK_VERSIONS: + print >> stderr, "Don't know about Spark version: {v}".format(v=version) + sys.exit(1) + return version + else: + github_commit_url = "{repo}/commit/{commit_hash}".format(repo=repo, commit_hash=version) + request = urllib2.Request(github_commit_url) + request.get_method = lambda: 'HEAD' + try: + response = urllib2.urlopen(request) + except urllib2.HTTPError, e: + print >> stderr, "Couldn't validate Spark commit: {url}".format(url=github_commit_url) + print >> stderr, "Received HTTP response code of {code}.".format(code=e.code) + sys.exit(1) + return version # Check whether a given EC2 instance object is in a state we consider active, @@ -203,26 +279,6 @@ def is_active(instance): return (instance.state in ['pending', 'running', 'stopping', 'stopped']) -# Return correct versions of Spark and Shark, given the supplied Spark version -def get_spark_shark_version(opts): - spark_shark_map = { - "0.7.3": "0.7.1", - "0.8.0": "0.8.0", - "0.8.1": "0.8.1", - "0.9.0": "0.9.0", - "0.9.1": "0.9.1", - "1.0.0": "1.0.0", - "1.0.1": "1.0.1", - "1.0.2": "1.0.2", - "1.1.0": "1.1.0", - } - version = opts.spark_version.replace("v", "") - if version not in spark_shark_map: - print >> stderr, "Don't know about Spark version: %s" % version - sys.exit(1) - return (version, spark_shark_map[version]) - - # Attempt to resolve an appropriate AMI given the architecture and region of the request. # Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ # Last Updated: 2014-06-20 @@ -303,12 +359,26 @@ def launch_cluster(conn, opts, cluster_name): user_data_content = user_data_file.read() print "Setting up security groups..." - master_group = get_or_make_group(conn, cluster_name + "-master") - slave_group = get_or_make_group(conn, cluster_name + "-slaves") + master_group = get_or_make_group(conn, cluster_name + "-master", opts.vpc_id) + slave_group = get_or_make_group(conn, cluster_name + "-slaves", opts.vpc_id) authorized_address = opts.authorized_address if master_group.rules == []: # Group was just now created - master_group.authorize(src_group=master_group) - master_group.authorize(src_group=slave_group) + if opts.vpc_id is None: + master_group.authorize(src_group=master_group) + master_group.authorize(src_group=slave_group) + else: + master_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, + src_group=master_group) + master_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, + src_group=master_group) + master_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, + src_group=master_group) + master_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, + src_group=slave_group) + master_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, + src_group=slave_group) + master_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, + src_group=slave_group) master_group.authorize('tcp', 22, 22, authorized_address) master_group.authorize('tcp', 8080, 8081, authorized_address) master_group.authorize('tcp', 18080, 18080, authorized_address) @@ -320,8 +390,22 @@ def launch_cluster(conn, opts, cluster_name): if opts.ganglia: master_group.authorize('tcp', 5080, 5080, authorized_address) if slave_group.rules == []: # Group was just now created - slave_group.authorize(src_group=master_group) - slave_group.authorize(src_group=slave_group) + if opts.vpc_id is None: + slave_group.authorize(src_group=master_group) + slave_group.authorize(src_group=slave_group) + else: + slave_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, + src_group=master_group) + slave_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, + src_group=master_group) + slave_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, + src_group=master_group) + slave_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, + src_group=slave_group) + slave_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, + src_group=slave_group) + slave_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, + src_group=slave_group) slave_group.authorize('tcp', 22, 22, authorized_address) slave_group.authorize('tcp', 8080, 8081, authorized_address) slave_group.authorize('tcp', 50060, 50060, authorized_address) @@ -341,11 +425,12 @@ def launch_cluster(conn, opts, cluster_name): if opts.ami is None: opts.ami = get_spark_ami(opts) - additional_groups = [] + # we use group ids to work around https://github.com/boto/boto/issues/350 + additional_group_ids = [] if opts.additional_security_group: - additional_groups = [sg - for sg in conn.get_all_security_groups() - if opts.additional_security_group in (sg.name, sg.id)] + additional_group_ids = [sg.id + for sg in conn.get_all_security_groups() + if opts.additional_security_group in (sg.name, sg.id)] print "Launching instances..." try: @@ -392,9 +477,11 @@ def launch_cluster(conn, opts, cluster_name): placement=zone, count=num_slaves_this_zone, key_name=opts.key_pair, - security_groups=[slave_group] + additional_groups, + security_group_ids=[slave_group.id] + additional_group_ids, instance_type=opts.instance_type, block_device_map=block_map, + subnet_id=opts.subnet_id, + placement_group=opts.placement_group, user_data=user_data_content) my_req_ids += [req.id for req in slave_reqs] i += 1 @@ -413,7 +500,7 @@ def launch_cluster(conn, opts, cluster_name): active_instance_ids.append(id_to_req[i].instance_id) if len(active_instance_ids) == opts.slaves: print "All %d slaves granted" % opts.slaves - reservations = conn.get_all_instances(active_instance_ids) + reservations = conn.get_all_reservations(active_instance_ids) slave_nodes = [] for r in reservations: slave_nodes += r.instances @@ -441,12 +528,14 @@ def launch_cluster(conn, opts, cluster_name): num_slaves_this_zone = get_partition(opts.slaves, num_zones, i) if num_slaves_this_zone > 0: slave_res = image.run(key_name=opts.key_pair, - security_groups=[slave_group] + additional_groups, + security_group_ids=[slave_group.id] + additional_group_ids, instance_type=opts.instance_type, placement=zone, min_count=num_slaves_this_zone, max_count=num_slaves_this_zone, block_device_map=block_map, + subnet_id=opts.subnet_id, + placement_group=opts.placement_group, user_data=user_data_content) slave_nodes += slave_res.instances print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone, @@ -467,13 +556,16 @@ def launch_cluster(conn, opts, cluster_name): if opts.zone == 'all': opts.zone = random.choice(conn.get_all_zones()).name master_res = image.run(key_name=opts.key_pair, - security_groups=[master_group] + additional_groups, + security_group_ids=[master_group.id] + additional_group_ids, instance_type=master_type, placement=opts.zone, min_count=1, max_count=1, block_device_map=block_map, + subnet_id=opts.subnet_id, + placement_group=opts.placement_group, user_data=user_data_content) + master_nodes = master_res.instances print "Launched master in %s, regid = %s" % (zone, master_res.id) @@ -497,7 +589,7 @@ def launch_cluster(conn, opts, cluster_name): def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): print "Searching for existing cluster " + cluster_name + "..." - reservations = conn.get_all_instances() + reservations = conn.get_all_reservations() master_nodes = [] slave_nodes = [] for res in reservations: @@ -540,7 +632,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): print slave.public_dns_name ssh_write(slave.public_dns_name, opts, ['tar', 'x'], dot_ssh_tar) - modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs', + modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', 'mapreduce', 'spark-standalone', 'tachyon'] if opts.hadoop_major_version == "1": @@ -574,12 +666,6 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): print "Done!" -def setup_standalone_cluster(master, slave_nodes, opts): - slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes]) - ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips)) - ssh(master, opts, "/root/spark/sbin/start-all.sh") - - def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") @@ -633,9 +719,7 @@ def wait_for_cluster_state(conn, opts, cluster_instances, cluster_state): sys.stdout.flush() start_time = datetime.now() - num_attempts = 0 - conn = ec2.connect_to_region(opts.region) while True: time.sleep(5 * num_attempts) # seconds @@ -742,13 +826,11 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): cluster_url = "%s:7077" % active_master if "." in opts.spark_version: - # Pre-built spark & shark deploy - (spark_v, shark_v) = get_spark_shark_version(opts) + # Pre-built Spark deploy + spark_v = get_validate_spark_version(opts.spark_version, opts.spark_git_repo) else: # Spark-only custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) - shark_v = "" - modules = filter(lambda x: x != "shark", modules) template_vars = { "master_list": '\n'.join([i.public_dns_name for i in master_nodes]), @@ -761,7 +843,6 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): "swap": str(opts.swap), "modules": '\n'.join(modules), "spark_version": spark_v, - "shark_version": shark_v, "hadoop_major_version": opts.hadoop_major_version, "spark_worker_instances": "%d" % opts.worker_instances, "spark_master_opts": opts.master_opts @@ -910,6 +991,8 @@ def real_main(): (opts, action, cluster_name) = parse_args() # Input parameter validation + get_validate_spark_version(opts.spark_version, opts.spark_git_repo) + if opts.wait is not None: # NOTE: DeprecationWarnings are silent in 2.7+ by default. # To show them, run Python with the -Wdefault switch. diff --git a/ec2/third_party/boto-2.4.1.zip b/ec2/third_party/boto-2.4.1.zip deleted file mode 100644 index 49886b89aeaea..0000000000000 Binary files a/ec2/third_party/boto-2.4.1.zip and /dev/null differ diff --git a/examples/pom.xml b/examples/pom.xml index 8713230e1e8ed..002d4458c4b3e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -98,143 +98,145 @@ ${project.version} - org.eclipse.jetty - jetty-server + org.apache.hbase + hbase-testing-util + ${hbase.version} + ${hbase.deps.scope} + + + + org.apache.hbase + hbase-annotations + + + org.jruby + jruby-complete + + + + + org.apache.hbase + hbase-protocol + ${hbase.version} + ${hbase.deps.scope} + + + org.apache.hbase + hbase-common + ${hbase.version} + ${hbase.deps.scope} + + + + org.apache.hbase + hbase-annotations + + + + + org.apache.hbase + hbase-client + ${hbase.version} + ${hbase.deps.scope} + + + + org.apache.hbase + hbase-annotations + + + io.netty + netty + + + + + org.apache.hbase + hbase-server + ${hbase.version} + ${hbase.deps.scope} + + + + org.apache.hbase + hbase-annotations + + + org.apache.hadoop + hadoop-core + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-mapreduce-client-jobclient + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-annotations + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hbase + hbase-hadoop1-compat + + + org.apache.commons + commons-math + + + com.sun.jersey + jersey-core + + + org.slf4j + slf4j-api + + + com.sun.jersey + jersey-server + + + com.sun.jersey + jersey-core + + + com.sun.jersey + jersey-json + + + + commons-io + commons-io + + + + + org.apache.hbase + hbase-hadoop-compat + ${hbase.version} + ${hbase.deps.scope} + + + org.apache.hbase + hbase-hadoop-compat + ${hbase.version} + test-jar + test - - org.apache.hbase - hbase-testing-util - ${hbase.version} - - - - org.apache.hbase - hbase-annotations - - - org.jruby - jruby-complete - - - - - org.apache.hbase - hbase-protocol - ${hbase.version} - - - org.apache.hbase - hbase-common - ${hbase.version} - - - - org.apache.hbase - hbase-annotations - - - - - org.apache.hbase - hbase-client - ${hbase.version} - - - - org.apache.hbase - hbase-annotations - - - io.netty - netty - - - - - org.apache.hbase - hbase-server - ${hbase.version} - - - org.apache.hadoop - hadoop-core - - - org.apache.hadoop - hadoop-client - - - org.apache.hadoop - hadoop-mapreduce-client-jobclient - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - org.apache.hadoop - hadoop-auth - - - - org.apache.hbase - hbase-annotations - - - org.apache.hadoop - hadoop-annotations - - - org.apache.hadoop - hadoop-hdfs - - - org.apache.hbase - hbase-hadoop1-compat - - - org.apache.commons - commons-math - - - com.sun.jersey - jersey-core - - - org.slf4j - slf4j-api - - - com.sun.jersey - jersey-server - - - com.sun.jersey - jersey-core - - - com.sun.jersey - jersey-json - - - - commons-io - commons-io - - - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - test-jar - test - org.apache.commons commons-math3 @@ -244,11 +246,6 @@ algebird-core_${scala.binary.version} 0.8.1 - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -313,31 +310,6 @@ org.apache.maven.plugins maven-shade-plugin - - false - ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar - - - *:* - - - - - com.google.guava:guava - - com/google/common/base/Optional* - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - package @@ -345,6 +317,34 @@ shade + false + ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar + + + *:* + + + + + com.google.guava:guava + + + ** + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + com.google @@ -416,7 +416,7 @@ - scala-2.10 @@ -454,5 +454,37 @@ + + + + flume-provided + + provided + + + + hadoop-provided + + provided + + + + hbase-provided + + provided + + + + hive-provided + + provided + + + + parquet-provided + + provided + + diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index c9e1511278ede..2adc63f7ff30e 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -22,7 +22,6 @@ import java.util.Properties import kafka.producer._ import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.kafka._ import org.apache.spark.SparkConf diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index 973049b95a7bd..1b53f3edbe92e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -28,20 +28,15 @@ object BroadcastTest { val bcName = if (args.length > 2) args(2) else "Http" val blockSize = if (args.length > 3) args(3) else "4096" - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast." + bcName + - "BroadcastFactory") - System.setProperty("spark.broadcast.blockSize", blockSize) val sparkConf = new SparkConf().setAppName("Broadcast Test") - + .set("spark.broadcast.factory", s"org.apache.spark.broadcast.${bcName}BroaddcastFactory") + .set("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(sparkConf) val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 - val arr1 = new Array[Int](num) - for (i <- 0 until arr1.length) { - arr1(i) = i - } + val arr1 = (0 until num).toArray for (i <- 0 until 3) { println("Iteration " + i) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index 828cffb01ca1e..409721b01c8fd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -33,6 +33,10 @@ object Analytics extends Logging { if (args.length < 2) { System.err.println( "Usage: Analytics --numEPart= [other options]") + System.err.println("Supported 'taskType' as follows:") + System.err.println(" pagerank Compute PageRank") + System.err.println(" cc Compute the connected components of vertices") + System.err.println(" triangles Count the number of triangles") System.exit(1) } @@ -46,7 +50,7 @@ object Analytics extends Logging { } val options = mutable.Map(optionsList: _*) - val conf = new SparkConf().set("spark.locality.wait", "100000") + val conf = new SparkConf() GraphXUtils.registerKryoClasses(conf) val numEPart = options.remove("numEPart").map(_.toInt).getOrElse { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala new file mode 100644 index 0000000000000..948c350953e27 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala @@ -0,0 +1,67 @@ +/* + * 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.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.clustering.GaussianMixtureEM +import org.apache.spark.mllib.linalg.Vectors + +/** + * An example Gaussian Mixture Model EM app. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.DenseGmmEM + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DenseGmmEM { + def main(args: Array[String]): Unit = { + if (args.length < 3) { + println("usage: DenseGmmEM [maxIterations]") + } else { + val maxIterations = if (args.length > 3) args(3).toInt else 100 + run(args(0), args(1).toInt, args(2).toDouble, maxIterations) + } + } + + private def run(inputFile: String, k: Int, convergenceTol: Double, maxIterations: Int) { + val conf = new SparkConf().setAppName("Gaussian Mixture Model EM example") + val ctx = new SparkContext(conf) + + val data = ctx.textFile(inputFile).map { line => + Vectors.dense(line.trim.split(' ').map(_.toDouble)) + }.cache() + + val clusters = new GaussianMixtureEM() + .setK(k) + .setConvergenceTol(convergenceTol) + .setMaxIterations(maxIterations) + .run(data) + + for (i <- 0 until clusters.k) { + println("weight=%f\nmu=%s\nsigma=\n%s\n" format + (clusters.weight(i), clusters.mu(i), clusters.sigma(i))) + } + + println("Cluster labels (first <= 100):") + val clusterLabels = clusters.predict(data) + clusterLabels.take(100).foreach { x => + print(" " + x) + } + println() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index 6bb659fbd8be8..30269a7ccae97 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -23,7 +23,6 @@ import java.net.Socket import org.apache.spark.{SparkConf, Logging} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.receiver.Receiver /** diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index 6c24bc3ad09e0..4b4667fec44e6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ /** * Counts words in new text files created in the given directory diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index e4283e04a1b11..6ff0c47793a25 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -22,7 +22,6 @@ import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.mqtt._ import org.apache.spark.SparkConf diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index ae0a08c6cdb1a..2cd8073dada14 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.storage.StorageLevel /** diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala index 4caa90659111a..13ba9a43ec3c9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.SynchronizedQueue import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ object QueueStream { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 19427e629f76d..c3a05c89d817e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -25,7 +25,6 @@ import com.google.common.io.Files import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Time, Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.util.IntParam /** diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index ed186ea5650c4..345d0bc441351 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -20,7 +20,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.HashPartitioner import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ /** * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala index 683752ac96241..62f49530edb12 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala @@ -23,7 +23,6 @@ import org.apache.spark.SparkConf import org.apache.spark.SparkContext._ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.twitter._ // scalastyle:off diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala index f55d23ab3924b..f253d75b279f7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala @@ -18,7 +18,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} -import StreamingContext._ import org.apache.spark.SparkContext._ import org.apache.spark.streaming.twitter._ import org.apache.spark.SparkConf diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index 79905af381a12..6510c70bd1866 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -24,7 +24,6 @@ import akka.zeromq.Subscribe import akka.util.ByteString import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.zeromq._ import scala.language.implicitConversions diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index 55226c0a6df60..fbacaee98690f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming.clickstream import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.examples.streaming.StreamingExamples // scalastyle:off /** Analyses a streaming dataset of web page views. This class demonstrates several types of diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 72618b6515f83..0706f1ebf66e2 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -38,37 +38,10 @@ org.apache.flume flume-ng-sdk - ${flume.version} - - - io.netty - netty - - - org.apache.thrift - libthrift - - org.apache.flume flume-ng-core - ${flume.version} - - - io.netty - netty - - - org.apache.thrift - libthrift - - - - - org.scalatest - scalatest_${scala.binary.version} - test org.scala-lang @@ -91,10 +64,6 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - org.scalatest - scalatest-maven-plugin - org.apache.avro avro-maven-plugin diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties index 4411d6e20c52a..2a58e99817224 100644 --- a/external/flume-sink/src/test/resources/log4j.properties +++ b/external/flume-sink/src/test/resources/log4j.properties @@ -17,9 +17,8 @@ # Set everything to be logged to the file streaming/target/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender 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/external/flume/pom.xml b/external/flume/pom.xml index a682f0e8471d8..1f2681394c583 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -48,23 +48,11 @@ org.apache.flume - flume-ng-sdk - ${flume.version} - - - io.netty - netty - - - org.apache.thrift - libthrift - - + flume-ng-core - org.scalatest - scalatest_${scala.binary.version} - test + org.apache.flume + flume-ng-sdk org.scalacheck @@ -85,11 +73,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index 4411d6e20c52a..9697237bfa1a3 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/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=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/external/kafka/pom.xml b/external/kafka/pom.xml index b3f44471cd326..b29b0509656ba 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -74,11 +74,6 @@ 3.2 test - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -98,11 +93,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index b4ac929e0c070..df725f0c65a64 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -75,7 +75,7 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages form a Kafka Broker. + * Create an input stream that pulls messages from a Kafka Broker. * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. * @param jssc JavaStreamingContext object * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) @@ -93,7 +93,7 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages form a Kafka Broker. + * Create an input stream that pulls messages from a Kafka Broker. * @param jssc JavaStreamingContext object * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. @@ -113,7 +113,7 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages form a Kafka Broker. + * Create an input stream that pulls messages from a Kafka Broker. * @param jssc JavaStreamingContext object * @param keyTypeClass Key type of RDD * @param valueTypeClass value type of RDD diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 4411d6e20c52a..9697237bfa1a3 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/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=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/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9025915f4447e..560c8b9d18276 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -46,11 +46,6 @@ org.eclipse.paho.client.mqttv3 1.0.1 - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -66,15 +61,15 @@ junit-interface test + + org.apache.activemq + activemq-core + 5.7.0 + test + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index 4411d6e20c52a..9697237bfa1a3 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/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=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/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 84595acf45ccb..98fe6cb301f52 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -17,31 +17,111 @@ package org.apache.spark.streaming.mqtt -import org.scalatest.FunSuite +import java.net.{URI, ServerSocket} -import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.activemq.broker.{TransportConnector, BrokerService} +import org.apache.spark.util.Utils +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually +import scala.concurrent.duration._ +import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.eclipse.paho.client.mqttv3._ +import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence -class MQTTStreamSuite extends FunSuite { - - val batchDuration = Seconds(1) +class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { + private val batchDuration = Milliseconds(500) private val master: String = "local[2]" - private val framework: String = this.getClass.getSimpleName + private val freePort = findFreePort() + private val brokerUri = "//localhost:" + freePort + private val topic = "def" + private var ssc: StreamingContext = _ + private val persistenceDir = Utils.createTempDir() + private var broker: BrokerService = _ + private var connector: TransportConnector = _ - test("mqtt input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) - val brokerUrl = "abc" - val topic = "def" + before { + ssc = new StreamingContext(master, framework, batchDuration) + setupMQTT() + } - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[String] = MQTTUtils.createStream(ssc, brokerUrl, topic) - val test2: ReceiverInputDStream[String] = - MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) + after { + if (ssc != null) { + ssc.stop() + ssc = null + } + Utils.deleteRecursively(persistenceDir) + tearDownMQTT() + } - // TODO: Actually test receiving data + test("mqtt input stream") { + val sendMessage = "MQTT demo for spark streaming" + val receiveStream: ReceiverInputDStream[String] = + MQTTUtils.createStream(ssc, "tcp:" + brokerUri, topic, StorageLevel.MEMORY_ONLY) + var receiveMessage: List[String] = List() + receiveStream.foreachRDD { rdd => + if (rdd.collect.length > 0) { + receiveMessage = receiveMessage ::: List(rdd.first) + receiveMessage + } + } + ssc.start() + publishData(sendMessage) + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { + assert(sendMessage.equals(receiveMessage(0))) + } ssc.stop() } + + private def setupMQTT() { + broker = new BrokerService() + connector = new TransportConnector() + connector.setName("mqtt") + connector.setUri(new URI("mqtt:" + brokerUri)) + broker.addConnector(connector) + broker.start() + } + + private def tearDownMQTT() { + if (broker != null) { + broker.stop() + broker = null + } + if (connector != null) { + connector.stop() + connector = null + } + } + + private def findFreePort(): Int = { + Utils.startServiceOnPort(23456, (trialPort: Int) => { + val socket = new ServerSocket(trialPort) + socket.close() + (null, trialPort) + })._2 + } + + def publishData(data: String): Unit = { + var client: MqttClient = null + try { + val persistence: MqttClientPersistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath) + client = new MqttClient("tcp:" + brokerUri, MqttClient.generateClientId(), persistence) + client.connect() + if (client.isConnected) { + val msgTopic: MqttTopic = client.getTopic(topic) + val message: MqttMessage = new MqttMessage(data.getBytes("utf-8")) + message.setQos(1) + message.setRetained(true) + for (i <- 0 to 100) + msgTopic.publish(message) + } + } finally { + client.disconnect() + client.close() + client = null + } + } } diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 000ace1446e5e..da6ffe7662f63 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -46,11 +46,6 @@ twitter4j-stream 3.0.3 - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -70,11 +65,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index 4411d6e20c52a..64bfc5745088f 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/target/unit-tests.log +# Set everything to be logged to the filetarget/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender 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/external/zeromq/pom.xml b/external/zeromq/pom.xml index 29c452093502e..e919c2c9b19ea 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -44,12 +44,6 @@ ${akka.group} akka-zeromq_${scala.binary.version} - ${akka.version} - - - org.scalatest - scalatest_${scala.binary.version} - test org.scalacheck @@ -70,11 +64,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 4411d6e20c52a..9697237bfa1a3 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/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=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/extras/java8-tests/README.md b/extras/java8-tests/README.md index e95b73ac7702a..dc9e87f2eeb92 100644 --- a/extras/java8-tests/README.md +++ b/extras/java8-tests/README.md @@ -8,7 +8,7 @@ to your Java location. The set-up depends a bit on the build system: `-java-home` to the sbt launch script. If a Java 8 JDK is detected sbt will automatically include the Java 8 test project. - `$ JAVA_HOME=/opt/jdk1.8.0/ sbt/sbt clean "test-only org.apache.spark.Java8APISuite"` + `$ JAVA_HOME=/opt/jdk1.8.0/ build/sbt clean "test-only org.apache.spark.Java8APISuite"` * For Maven users, @@ -19,6 +19,6 @@ to your Java location. The set-up depends a bit on the build system: `$ JAVA_HOME=/opt/jdk1.8.0/ mvn clean install -DskipTests` `$ JAVA_HOME=/opt/jdk1.8.0/ mvn test -Pjava8-tests -DwildcardSuites=org.apache.spark.Java8APISuite` - Note that the above command can only be run from project root directory since this module - depends on core and the test-jars of core and streaming. This means an install step is + Note that the above command can only be run from project root directory since this module + depends on core and the test-jars of core and streaming. This means an install step is required to make the test dependencies visible to the Java 8 sub-project. diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index c8477a6566311..0fb431808bacd 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -60,11 +60,6 @@ junit-interface test - - org.scalatest - scalatest_${scala.binary.version} - test - @@ -159,16 +154,6 @@ - - org.scalatest - scalatest-maven-plugin - - - test - none - - - diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties index bb0ab319a0080..287c8e3563503 100644 --- a/extras/java8-tests/src/test/resources/log4j.properties +++ b/extras/java8-tests/src/test/resources/log4j.properties @@ -18,7 +18,7 @@ # 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/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index c0d3a61119113..c815eda52bda7 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -57,11 +57,6 @@ aws-java-sdk ${aws.java.sdk.version} - - org.scalatest - scalatest_${scala.binary.version} - test - org.mockito mockito-all @@ -86,11 +81,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties index d9d08f68687d3..853ef0ed2986f 100644 --- a/extras/kinesis-asl/src/test/resources/log4j.properties +++ b/extras/kinesis-asl/src/test/resources/log4j.properties @@ -14,10 +14,11 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +# Set everything to be logged to the file target/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender 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/graphx/pom.xml b/graphx/pom.xml index 9982b36f9b62f..72374aae6da9b 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -45,15 +45,6 @@ jblas ${jblas.version} - - org.eclipse.jetty - jetty-server - - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -63,11 +54,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 116d1ea700175..dc8b4789c4b61 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -278,6 +278,32 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali retVal } + /** + * Convert bi-directional edges into uni-directional ones. + * Some graph algorithms (e.g., TriangleCount) assume that an input graph + * has its edges in canonical direction. + * This function rewrites the vertex ids of edges so that srcIds are bigger + * than dstIds, and merges the duplicated edges. + * + * @param mergeFunc the user defined reduce function which should + * be commutative and associative and is used to combine the output + * of the map phase + * + * @return the resulting graph with canonical edges + */ + def convertToCanonicalEdges( + mergeFunc: (ED, ED) => ED = (e1, e2) => e1): Graph[VD, ED] = { + val newEdges = + graph.edges + .map { + case e if e.srcId < e.dstId => ((e.srcId, e.dstId), e.attr) + case e => ((e.dstId, e.srcId), e.attr) + } + .reduceByKey(mergeFunc) + .map(e => new Edge(e._1._1, e._1._2, e._2)) + Graph(graph.vertices, newEdges) + } + /** * Execute a Pregel-like iterative vertex-parallel abstraction. The * user-defined vertex-program `vprog` is executed in parallel on diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 13033fee0e6b5..7372dfbd9fe98 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -32,9 +32,9 @@ trait PartitionStrategy extends Serializable { object PartitionStrategy { /** * Assigns edges to partitions using a 2D partitioning of the sparse edge adjacency matrix, - * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication. + * guaranteeing a `2 * sqrt(numParts) - 1` bound on vertex replication. * - * Suppose we have a graph with 11 vertices that we want to partition + * Suppose we have a graph with 12 vertices that we want to partition * over 9 machines. We can use the following sparse matrix representation: * *
@@ -61,7 +61,7 @@ object PartitionStrategy {
    * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3,
    * P6)` or the last
    * row of blocks `(P6, P7, P8)`.  As a consequence we can guarantee that `v11` will need to be
-   * replicated to at most `2 * sqrt(numParts)` machines.
+   * replicated to at most `2 * sqrt(numParts) - 1` machines.
    *
    * Notice that `P0` has many edges and as a consequence this partitioning would lead to poor work
    * balance.  To improve balance we first multiply each vertex id by a large prime to shuffle the
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
index 409cf60977f6f..906d42328fcb9 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
@@ -129,44 +129,45 @@ private[impl] case class EdgeWithLocalIds[@specialized ED](
     srcId: VertexId, dstId: VertexId, localSrcId: Int, localDstId: Int, attr: ED)
 
 private[impl] object EdgeWithLocalIds {
-  implicit def lexicographicOrdering[ED] = new Ordering[EdgeWithLocalIds[ED]] {
-    override def compare(a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]): Int = {
-      if (a.srcId == b.srcId) {
-        if (a.dstId == b.dstId) 0
-        else if (a.dstId < b.dstId) -1
+  implicit def lexicographicOrdering[ED]: Ordering[EdgeWithLocalIds[ED]] =
+    new Ordering[EdgeWithLocalIds[ED]] {
+      override def compare(a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]): Int = {
+        if (a.srcId == b.srcId) {
+          if (a.dstId == b.dstId) 0
+          else if (a.dstId < b.dstId) -1
+          else 1
+        } else if (a.srcId < b.srcId) -1
         else 1
-      } else if (a.srcId < b.srcId) -1
-      else 1
+      }
     }
-  }
 
-  private[graphx] def edgeArraySortDataFormat[ED]
-      = new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] {
-    override def getKey(
-        data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = {
-      data(pos)
-    }
+  private[graphx] def edgeArraySortDataFormat[ED] = {
+    new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] {
+      override def getKey(data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = {
+        data(pos)
+      }
 
-    override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = {
-      val tmp = data(pos0)
-      data(pos0) = data(pos1)
-      data(pos1) = tmp
-    }
+      override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = {
+        val tmp = data(pos0)
+        data(pos0) = data(pos1)
+        data(pos1) = tmp
+      }
 
-    override def copyElement(
-        src: Array[EdgeWithLocalIds[ED]], srcPos: Int,
-        dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) {
-      dst(dstPos) = src(srcPos)
-    }
+      override def copyElement(
+          src: Array[EdgeWithLocalIds[ED]], srcPos: Int,
+          dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) {
+        dst(dstPos) = src(srcPos)
+      }
 
-    override def copyRange(
-        src: Array[EdgeWithLocalIds[ED]], srcPos: Int,
-        dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) {
-      System.arraycopy(src, srcPos, dst, dstPos, length)
-    }
+      override def copyRange(
+          src: Array[EdgeWithLocalIds[ED]], srcPos: Int,
+          dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) {
+        System.arraycopy(src, srcPos, dst, dstPos, length)
+      }
 
-    override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = {
-      new Array[EdgeWithLocalIds[ED]](length)
+      override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = {
+        new Array[EdgeWithLocalIds[ED]](length)
+      }
     }
   }
 }
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala
index 5412d720475dc..aa320088f2088 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala
@@ -74,8 +74,8 @@ object ShippableVertexPartition {
    * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a
    * `ShippableVertexPartition`.
    */
-  implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) =
-    new ShippableVertexPartitionOps(partition)
+  implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD])
+    : ShippableVertexPartitionOps[VD] = new ShippableVertexPartitionOps(partition)
 
   /**
    * Implicit evidence that `ShippableVertexPartition` is a member of the
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala
index 55c7a19d1bdab..fbe53acfc32aa 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala
@@ -38,8 +38,8 @@ private[graphx] object VertexPartition {
    * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a
    * `VertexPartition`.
    */
-  implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) =
-    new VertexPartitionOps(partition)
+  implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD])
+    : VertexPartitionOps[VD] = new VertexPartitionOps(partition)
 
   /**
    * Implicit evidence that `VertexPartition` is a member of the `VertexPartitionBaseOpsConstructor`
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
index b40aa1b417a0f..4fd2548b7faf6 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
@@ -238,8 +238,8 @@ private[graphx] abstract class VertexPartitionBaseOps
    * because these methods return a `Self` and this implicit conversion re-wraps that in a
    * `VertexPartitionBaseOps`. This relies on the context bound on `Self`.
    */
-  private implicit def toOps[VD2: ClassTag](
-      partition: Self[VD2]): VertexPartitionBaseOps[VD2, Self] = {
+  private implicit def toOps[VD2: ClassTag](partition: Self[VD2])
+    : VertexPartitionBaseOps[VD2, Self] = {
     implicitly[VertexPartitionBaseOpsConstructor[Self]].toOps(partition)
   }
 }
diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties
index 9dd05f17f012b..287c8e3563503 100644
--- a/graphx/src/test/resources/log4j.properties
+++ b/graphx/src/test/resources/log4j.properties
@@ -15,10 +15,10 @@
 # limitations under the License.
 #
 
-# Set everything to be logged to the file core/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/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
index ea94d4accb63b..9bc8007ce49cd 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
@@ -79,6 +79,21 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext {
     }
   }
 
+  test ("convertToCanonicalEdges") {
+    withSpark { sc =>
+      val vertices =
+        sc.parallelize(Seq[(VertexId, String)]((1, "one"), (2, "two"), (3, "three")), 2)
+      val edges =
+        sc.parallelize(Seq(Edge(1, 2, 1), Edge(2, 1, 1), Edge(3, 2, 2)))
+      val g: Graph[String, Int] = Graph(vertices, edges)
+
+      val g1 = g.convertToCanonicalEdges()
+
+      val e = g1.edges.collect().toSet
+      assert(e === Set(Edge(1, 2, 1), Edge(2, 3, 2)))
+    }
+  }
+
   test("collectEdgesCycleDirectionOut") {
     withSpark { sc =>
       val graph = getCycleGraph(sc, 100)
diff --git a/make-distribution.sh b/make-distribution.sh
index 45c99e42e5a5b..4e2f400be3053 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -28,18 +28,20 @@ set -o pipefail
 set -e
 
 # Figure out where the Spark framework is installed
-FWDIR="$(cd "`dirname "$0"`"; pwd)"
-DISTDIR="$FWDIR/dist"
+SPARK_HOME="$(cd "`dirname "$0"`"; pwd)"
+DISTDIR="$SPARK_HOME/dist"
 
 SPARK_TACHYON=false
 MAKE_TGZ=false
 NAME=none
+MVN="$SPARK_HOME/build/mvn"
 
 function exit_with_usage {
   echo "make-distribution.sh - tool for making binary distributions of Spark"
   echo ""
   echo "usage:"
-  echo "./make-distribution.sh [--name] [--tgz] [--with-tachyon] "
+  cl_options="[--name] [--tgz] [--mvn ] [--with-tachyon]"
+  echo "./make-distribution.sh $cl_options "
   echo "See Spark's \"Building Spark\" doc for correct Maven options."
   echo ""
   exit 1
@@ -71,6 +73,10 @@ while (( "$#" )); do
     --tgz)
       MAKE_TGZ=true
       ;;
+    --mvn)
+      MVN="$2"
+      shift
+      ;;
     --name)
       NAME="$2"
       shift
@@ -109,9 +115,9 @@ if which git &>/dev/null; then
     unset GITREV
 fi
 
-if ! which mvn &>/dev/null; then
-    echo -e "You need Maven installed to build Spark."
-    echo -e "Download Maven from https://maven.apache.org/"
+if ! which $MVN &>/dev/null; then
+    echo -e "Could not locate Maven command: '$MVN'."
+    echo -e "Specify the Maven command with the --mvn flag"
     exit -1;
 fi
 
@@ -119,7 +125,7 @@ VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "
 SPARK_HADOOP_VERSION=$(mvn help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\
     | grep -v "INFO"\
     | tail -n 1)
-SPARK_HIVE=$(mvn help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\
+SPARK_HIVE=$($MVN help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\
     | grep -v "INFO"\
     | fgrep --count "hive";\
     # Reset exit status to 0, otherwise the script stops here if the last grep finds nothing\
@@ -161,11 +167,11 @@ else
 fi
 
 # Build uber fat JAR
-cd "$FWDIR"
+cd "$SPARK_HOME"
 
 export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"
 
-BUILD_COMMAND="mvn clean package -DskipTests $@"
+BUILD_COMMAND="$MVN clean package -DskipTests $@"
 
 # Actually build the jar
 echo -e "\nBuilding with..."
@@ -177,41 +183,42 @@ ${BUILD_COMMAND}
 rm -rf "$DISTDIR"
 mkdir -p "$DISTDIR/lib"
 echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE"
+echo "Build flags: $@" >> "$DISTDIR/RELEASE"
 
 # Copy jars
-cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
-cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
+cp "$SPARK_HOME"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
+cp "$SPARK_HOME"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
 # This will fail if the -Pyarn profile is not provided
 # In this case, silence the error and ignore the return code of this command
-cp "$FWDIR"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || :
+cp "$SPARK_HOME"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || :
 
 # Copy example sources (needed for python and SQL)
 mkdir -p "$DISTDIR/examples/src/main"
-cp -r "$FWDIR"/examples/src/main "$DISTDIR/examples/src/"
+cp -r "$SPARK_HOME"/examples/src/main "$DISTDIR/examples/src/"
 
 if [ "$SPARK_HIVE" == "1" ]; then
-  cp "$FWDIR"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/"
+  cp "$SPARK_HOME"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/"
 fi
 
 # Copy license and ASF files
-cp "$FWDIR/LICENSE" "$DISTDIR"
-cp "$FWDIR/NOTICE" "$DISTDIR"
+cp "$SPARK_HOME/LICENSE" "$DISTDIR"
+cp "$SPARK_HOME/NOTICE" "$DISTDIR"
 
-if [ -e "$FWDIR"/CHANGES.txt ]; then
-  cp "$FWDIR/CHANGES.txt" "$DISTDIR"
+if [ -e "$SPARK_HOME"/CHANGES.txt ]; then
+  cp "$SPARK_HOME/CHANGES.txt" "$DISTDIR"
 fi
 
 # Copy data files
-cp -r "$FWDIR/data" "$DISTDIR"
+cp -r "$SPARK_HOME/data" "$DISTDIR"
 
 # Copy other things
 mkdir "$DISTDIR"/conf
-cp "$FWDIR"/conf/*.template "$DISTDIR"/conf
-cp "$FWDIR/README.md" "$DISTDIR"
-cp -r "$FWDIR/bin" "$DISTDIR"
-cp -r "$FWDIR/python" "$DISTDIR"
-cp -r "$FWDIR/sbin" "$DISTDIR"
-cp -r "$FWDIR/ec2" "$DISTDIR"
+cp "$SPARK_HOME"/conf/*.template "$DISTDIR"/conf
+cp "$SPARK_HOME/README.md" "$DISTDIR"
+cp -r "$SPARK_HOME/bin" "$DISTDIR"
+cp -r "$SPARK_HOME/python" "$DISTDIR"
+cp -r "$SPARK_HOME/sbin" "$DISTDIR"
+cp -r "$SPARK_HOME/ec2" "$DISTDIR"
 
 # Download and copy in tachyon, if requested
 if [ "$SPARK_TACHYON" == "true" ]; then
@@ -243,9 +250,9 @@ fi
 
 if [ "$MAKE_TGZ" == "true" ]; then
   TARDIR_NAME=spark-$VERSION-bin-$NAME
-  TARDIR="$FWDIR/$TARDIR_NAME"
+  TARDIR="$SPARK_HOME/$TARDIR_NAME"
   rm -rf "$TARDIR"
   cp -r "$DISTDIR" "$TARDIR"
-  tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$FWDIR" "$TARDIR_NAME"
+  tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$SPARK_HOME" "$TARDIR_NAME"
   rm -rf "$TARDIR"
 fi
diff --git a/mllib/pom.xml b/mllib/pom.xml
index 0a6dda0ab8c80..a0bda89ccaa71 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -29,7 +29,7 @@
   spark-mllib_2.10
   
     mllib
-    
+  
   jar
   Spark Project ML Library
   http://spark.apache.org/
@@ -50,10 +50,6 @@
       spark-sql_${scala.binary.version}
       ${project.version}
     
-    
-      org.eclipse.jetty
-      jetty-server
-    
     
       org.jblas
       jblas
@@ -80,11 +76,6 @@
       org.apache.commons
       commons-math3
     
-    
-      org.scalatest
-      scalatest_${scala.binary.version}
-      test
-    
     
       org.scalacheck
       scalacheck_${scala.binary.version}
@@ -129,12 +120,6 @@
   
     target/scala-${scala.binary.version}/classes
     target/scala-${scala.binary.version}/test-classes
-    
-      
-        org.scalatest
-        scalatest-maven-plugin
-      
-    
     
       
         ../python
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
index c4e5fd8e461fc..555da8c7e7ab3 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -624,6 +624,21 @@ class PythonMLLibAPI extends Serializable {
     RG.normalRDD(jsc.sc, size, parts, s)
   }
 
+  /**
+   * Java stub for Python mllib RandomRDDGenerators.logNormalRDD()
+   */
+  def logNormalRDD(jsc: JavaSparkContext,
+      mean: Double,
+      std: Double,
+      size: Long,
+      numPartitions: java.lang.Integer,
+      seed: java.lang.Long): JavaRDD[Double] = {
+    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
+    val s = getSeedOrDefault(seed)
+    RG.logNormalRDD(jsc.sc, mean, std, size, parts, s)
+  }
+
+
   /**
    * Java stub for Python mllib RandomRDDGenerators.poissonRDD()
    */
@@ -637,6 +652,33 @@ class PythonMLLibAPI extends Serializable {
     RG.poissonRDD(jsc.sc, mean, size, parts, s)
   }
 
+  /**
+   * Java stub for Python mllib RandomRDDGenerators.exponentialRDD()
+   */
+  def exponentialRDD(jsc: JavaSparkContext,
+      mean: Double,
+      size: Long,
+      numPartitions: java.lang.Integer,
+      seed: java.lang.Long): JavaRDD[Double] = {
+    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
+    val s = getSeedOrDefault(seed)
+    RG.exponentialRDD(jsc.sc, mean, size, parts, s)
+  }
+
+  /**
+   * Java stub for Python mllib RandomRDDGenerators.gammaRDD()
+   */
+  def gammaRDD(jsc: JavaSparkContext,
+      shape: Double,
+      scale: Double,
+      size: Long,
+      numPartitions: java.lang.Integer,
+      seed: java.lang.Long): JavaRDD[Double] = {
+    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
+    val s = getSeedOrDefault(seed)
+    RG.gammaRDD(jsc.sc, shape, scale, size, parts, s)
+  }
+
   /**
    * Java stub for Python mllib RandomRDDGenerators.uniformVectorRDD()
    */
@@ -663,6 +705,22 @@ class PythonMLLibAPI extends Serializable {
     RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s)
   }
 
+  /**
+   * Java stub for Python mllib RandomRDDGenerators.logNormalVectorRDD()
+   */
+  def logNormalVectorRDD(jsc: JavaSparkContext,
+      mean: Double,
+      std: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: java.lang.Integer,
+      seed: java.lang.Long): JavaRDD[Vector] = {
+    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
+    val s = getSeedOrDefault(seed)
+    RG.logNormalVectorRDD(jsc.sc, mean, std, numRows, numCols, parts, s)
+  }
+
+
   /**
    * Java stub for Python mllib RandomRDDGenerators.poissonVectorRDD()
    */
@@ -677,6 +735,36 @@ class PythonMLLibAPI extends Serializable {
     RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s)
   }
 
+  /**
+   * Java stub for Python mllib RandomRDDGenerators.exponentialVectorRDD()
+   */
+  def exponentialVectorRDD(jsc: JavaSparkContext,
+      mean: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: java.lang.Integer,
+      seed: java.lang.Long): JavaRDD[Vector] = {
+    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
+    val s = getSeedOrDefault(seed)
+    RG.exponentialVectorRDD(jsc.sc, mean, numRows, numCols, parts, s)
+  }
+
+  /**
+   * Java stub for Python mllib RandomRDDGenerators.gammaVectorRDD()
+   */
+  def gammaVectorRDD(jsc: JavaSparkContext,
+      shape: Double,
+      scale: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: java.lang.Integer,
+      seed: java.lang.Long): JavaRDD[Vector] = {
+    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
+    val s = getSeedOrDefault(seed)
+    RG.gammaVectorRDD(jsc.sc, shape, scale, numRows, numCols, parts, s)
+  }
+
+
 }
 
 /**
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
index 8c8e4a161aa5b..a967df857bed3 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
@@ -93,10 +93,10 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with
   def run(data: RDD[LabeledPoint]) = {
     val requireNonnegativeValues: Vector => Unit = (v: Vector) => {
       val values = v match {
-        case sv: SparseVector =>
-          sv.values
-        case dv: DenseVector =>
-          dv.values
+        case SparseVector(size, indices, values) =>
+          values
+        case DenseVector(values) =>
+          values
       }
       if (!values.forall(_ >= 0.0)) {
         throw new SparkException(s"Naive Bayes requires nonnegative feature values but found $v.")
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala
new file mode 100644
index 0000000000000..3a6c0e681e3fa
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala
@@ -0,0 +1,243 @@
+/*
+ * 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.mllib.clustering
+
+import scala.collection.mutable.IndexedSeq
+
+import breeze.linalg.{DenseVector => BreezeVector, DenseMatrix => BreezeMatrix, diag, Transpose}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors, DenseVector, DenseMatrix, BLAS}
+import org.apache.spark.mllib.stat.impl.MultivariateGaussian
+import org.apache.spark.mllib.util.MLUtils
+
+/**
+ * This class performs expectation maximization for multivariate Gaussian
+ * Mixture Models (GMMs).  A GMM represents a composite distribution of
+ * independent Gaussian distributions with associated "mixing" weights
+ * specifying each's contribution to the composite.
+ *
+ * Given a set of sample points, this class will maximize the log-likelihood 
+ * for a mixture of k Gaussians, iterating until the log-likelihood changes by 
+ * less than convergenceTol, or until it has reached the max number of iterations.
+ * While this process is generally guaranteed to converge, it is not guaranteed
+ * to find a global optimum.  
+ * 
+ * @param k The number of independent Gaussians in the mixture model
+ * @param convergenceTol The maximum change in log-likelihood at which convergence
+ * is considered to have occurred.
+ * @param maxIterations The maximum number of iterations to perform
+ */
+class GaussianMixtureEM private (
+    private var k: Int, 
+    private var convergenceTol: Double, 
+    private var maxIterations: Int) extends Serializable {
+  
+  /** A default instance, 2 Gaussians, 100 iterations, 0.01 log-likelihood threshold */
+  def this() = this(2, 0.01, 100)
+  
+  // number of samples per cluster to use when initializing Gaussians
+  private val nSamples = 5
+  
+  // an initializing GMM can be provided rather than using the 
+  // default random starting point
+  private var initialModel: Option[GaussianMixtureModel] = None
+  
+  /** Set the initial GMM starting point, bypassing the random initialization.
+   *  You must call setK() prior to calling this method, and the condition
+   *  (model.k == this.k) must be met; failure will result in an IllegalArgumentException
+   */
+  def setInitialModel(model: GaussianMixtureModel): this.type = {
+    if (model.k == k) {
+      initialModel = Some(model)
+    } else {
+      throw new IllegalArgumentException("mismatched cluster count (model.k != k)")
+    }
+    this
+  }
+  
+  /** Return the user supplied initial GMM, if supplied */
+  def getInitialModel: Option[GaussianMixtureModel] = initialModel
+  
+  /** Set the number of Gaussians in the mixture model.  Default: 2 */
+  def setK(k: Int): this.type = {
+    this.k = k
+    this
+  }
+  
+  /** Return the number of Gaussians in the mixture model */
+  def getK: Int = k
+  
+  /** Set the maximum number of iterations to run. Default: 100 */
+  def setMaxIterations(maxIterations: Int): this.type = {
+    this.maxIterations = maxIterations
+    this
+  }
+  
+  /** Return the maximum number of iterations to run */
+  def getMaxIterations: Int = maxIterations
+  
+  /**
+   * Set the largest change in log-likelihood at which convergence is 
+   * considered to have occurred.
+   */
+  def setConvergenceTol(convergenceTol: Double): this.type = {
+    this.convergenceTol = convergenceTol
+    this
+  }
+  
+  /** Return the largest change in log-likelihood at which convergence is
+   *  considered to have occurred.
+   */
+  def getConvergenceTol: Double = convergenceTol
+  
+  /** Perform expectation maximization */
+  def run(data: RDD[Vector]): GaussianMixtureModel = {
+    val sc = data.sparkContext
+    
+    // we will operate on the data as breeze data
+    val breezeData = data.map(u => u.toBreeze.toDenseVector).cache()
+    
+    // Get length of the input vectors
+    val d = breezeData.first.length 
+    
+    // Determine initial weights and corresponding Gaussians.
+    // If the user supplied an initial GMM, we use those values, otherwise
+    // we start with uniform weights, a random mean from the data, and
+    // diagonal covariance matrices using component variances
+    // derived from the samples    
+    val (weights, gaussians) = initialModel match {
+      case Some(gmm) => (gmm.weight, gmm.mu.zip(gmm.sigma).map { case(mu, sigma) => 
+        new MultivariateGaussian(mu.toBreeze.toDenseVector, sigma.toBreeze.toDenseMatrix) 
+      })
+      
+      case None => {
+        val samples = breezeData.takeSample(true, k * nSamples, scala.util.Random.nextInt)
+        (Array.fill(k)(1.0 / k), Array.tabulate(k) { i => 
+          val slice = samples.view(i * nSamples, (i + 1) * nSamples)
+          new MultivariateGaussian(vectorMean(slice), initCovariance(slice)) 
+        })  
+      }
+    }
+    
+    var llh = Double.MinValue // current log-likelihood 
+    var llhp = 0.0            // previous log-likelihood
+    
+    var iter = 0
+    while(iter < maxIterations && Math.abs(llh-llhp) > convergenceTol) {
+      // create and broadcast curried cluster contribution function
+      val compute = sc.broadcast(ExpectationSum.add(weights, gaussians)_)
+      
+      // aggregate the cluster contribution for all sample points
+      val sums = breezeData.aggregate(ExpectationSum.zero(k, d))(compute.value, _ += _)
+      
+      // Create new distributions based on the partial assignments
+      // (often referred to as the "M" step in literature)
+      val sumWeights = sums.weights.sum
+      var i = 0
+      while (i < k) {
+        val mu = sums.means(i) / sums.weights(i)
+        BLAS.syr(-sums.weights(i), Vectors.fromBreeze(mu).asInstanceOf[DenseVector],
+          Matrices.fromBreeze(sums.sigmas(i)).asInstanceOf[DenseMatrix])
+        weights(i) = sums.weights(i) / sumWeights
+        gaussians(i) = new MultivariateGaussian(mu, sums.sigmas(i) / sums.weights(i))
+        i = i + 1
+      }
+   
+      llhp = llh // current becomes previous
+      llh = sums.logLikelihood // this is the freshly computed log-likelihood
+      iter += 1
+    } 
+    
+    // Need to convert the breeze matrices to MLlib matrices
+    val means = Array.tabulate(k) { i => Vectors.fromBreeze(gaussians(i).mu) }
+    val sigmas = Array.tabulate(k) { i => Matrices.fromBreeze(gaussians(i).sigma) }
+    new GaussianMixtureModel(weights, means, sigmas)
+  }
+    
+  /** Average of dense breeze vectors */
+  private def vectorMean(x: IndexedSeq[BreezeVector[Double]]): BreezeVector[Double] = {
+    val v = BreezeVector.zeros[Double](x(0).length)
+    x.foreach(xi => v += xi)
+    v / x.length.toDouble 
+  }
+  
+  /**
+   * Construct matrix where diagonal entries are element-wise
+   * variance of input vectors (computes biased variance)
+   */
+  private def initCovariance(x: IndexedSeq[BreezeVector[Double]]): BreezeMatrix[Double] = {
+    val mu = vectorMean(x)
+    val ss = BreezeVector.zeros[Double](x(0).length)
+    x.map(xi => (xi - mu) :^ 2.0).foreach(u => ss += u)
+    diag(ss / x.length.toDouble)
+  }
+}
+
+// companion class to provide zero constructor for ExpectationSum
+private object ExpectationSum {
+  def zero(k: Int, d: Int): ExpectationSum = {
+    new ExpectationSum(0.0, Array.fill(k)(0.0), 
+      Array.fill(k)(BreezeVector.zeros(d)), Array.fill(k)(BreezeMatrix.zeros(d,d)))
+  }
+  
+  // compute cluster contributions for each input point
+  // (U, T) => U for aggregation
+  def add(
+      weights: Array[Double], 
+      dists: Array[MultivariateGaussian])
+      (sums: ExpectationSum, x: BreezeVector[Double]): ExpectationSum = {
+    val p = weights.zip(dists).map {
+      case (weight, dist) => MLUtils.EPSILON + weight * dist.pdf(x)
+    }
+    val pSum = p.sum
+    sums.logLikelihood += math.log(pSum)
+    val xxt = x * new Transpose(x)
+    var i = 0
+    while (i < sums.k) {
+      p(i) /= pSum
+      sums.weights(i) += p(i)
+      sums.means(i) += x * p(i)
+      BLAS.syr(p(i), Vectors.fromBreeze(x).asInstanceOf[DenseVector],
+        Matrices.fromBreeze(sums.sigmas(i)).asInstanceOf[DenseMatrix])
+      i = i + 1
+    }
+    sums
+  }  
+}
+
+// Aggregation class for partial expectation results
+private class ExpectationSum(
+    var logLikelihood: Double,
+    val weights: Array[Double],
+    val means: Array[BreezeVector[Double]],
+    val sigmas: Array[BreezeMatrix[Double]]) extends Serializable {
+  
+  val k = weights.length
+  
+  def +=(x: ExpectationSum): ExpectationSum = {
+    var i = 0
+    while (i < k) {
+      weights(i) += x.weights(i)
+      means(i) += x.means(i)
+      sigmas(i) += x.sigmas(i)
+      i = i + 1
+    }
+    logLikelihood += x.logLikelihood
+    this
+  }  
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala
new file mode 100644
index 0000000000000..b461ea4f0f06e
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala
@@ -0,0 +1,86 @@
+/*
+ * 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.mllib.clustering
+
+import breeze.linalg.{DenseVector => BreezeVector}
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.{Matrix, Vector}
+import org.apache.spark.mllib.stat.impl.MultivariateGaussian
+import org.apache.spark.mllib.util.MLUtils
+
+/**
+ * Multivariate Gaussian Mixture Model (GMM) consisting of k Gaussians, where points 
+ * are drawn from each Gaussian i=1..k with probability w(i); mu(i) and sigma(i) are 
+ * the respective mean and covariance for each Gaussian distribution i=1..k. 
+ * 
+ * @param weight Weights for each Gaussian distribution in the mixture, where weight(i) is
+ *               the weight for Gaussian i, and weight.sum == 1
+ * @param mu Means for each Gaussian in the mixture, where mu(i) is the mean for Gaussian i
+ * @param sigma Covariance maxtrix for each Gaussian in the mixture, where sigma(i) is the
+ *              covariance matrix for Gaussian i
+ */
+class GaussianMixtureModel(
+  val weight: Array[Double], 
+  val mu: Array[Vector], 
+  val sigma: Array[Matrix]) extends Serializable {
+  
+  /** Number of gaussians in mixture */
+  def k: Int = weight.length
+
+  /** Maps given points to their cluster indices. */
+  def predict(points: RDD[Vector]): RDD[Int] = {
+    val responsibilityMatrix = predictSoft(points)
+    responsibilityMatrix.map(r => r.indexOf(r.max))
+  }
+  
+  /**
+   * Given the input vectors, return the membership value of each vector
+   * to all mixture components. 
+   */
+  def predictSoft(points: RDD[Vector]): RDD[Array[Double]] = {
+    val sc = points.sparkContext
+    val dists = sc.broadcast {
+      (0 until k).map { i => 
+        new MultivariateGaussian(mu(i).toBreeze.toDenseVector, sigma(i).toBreeze.toDenseMatrix)
+      }.toArray
+    }
+    val weights = sc.broadcast(weight)
+    points.map { x => 
+      computeSoftAssignments(x.toBreeze.toDenseVector, dists.value, weights.value, k)
+    }
+  }
+  
+  /**
+   * Compute the partial assignments for each vector
+   */
+  private def computeSoftAssignments(
+      pt: BreezeVector[Double],
+      dists: Array[MultivariateGaussian],
+      weights: Array[Double],
+      k: Int): Array[Double] = {
+    val p = weights.zip(dists).map {
+      case (weight, dist) => MLUtils.EPSILON + weight * dist.pdf(pt)
+    }
+    val pSum = p.sum 
+    for (i <- 0 until k) {
+      p(i) /= pSum
+    }
+    p
+  }  
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala
index 6189dce9b27da..7752c1988fdd1 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala
@@ -24,7 +24,6 @@ import org.apache.spark.SparkContext._
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors}
 import org.apache.spark.rdd.RDD
-import org.apache.spark.streaming.StreamingContext._
 import org.apache.spark.streaming.dstream.DStream
 import org.apache.spark.util.Utils
 import org.apache.spark.util.random.XORShiftRandom
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
index 1af40de2c7fcf..ced042e2f96ca 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
@@ -28,9 +28,30 @@ import org.apache.spark.rdd.{RDD, UnionRDD}
  * Evaluator for binary classification.
  *
  * @param scoreAndLabels an RDD of (score, label) pairs.
+ * @param numBins if greater than 0, then the curves (ROC curve, PR curve) computed internally
+ *                will be down-sampled to this many "bins". If 0, no down-sampling will occur.
+ *                This is useful because the curve contains a point for each distinct score
+ *                in the input, and this could be as large as the input itself -- millions of
+ *                points or more, when thousands may be entirely sufficient to summarize
+ *                the curve. After down-sampling, the curves will instead be made of approximately
+ *                `numBins` points instead. Points are made from bins of equal numbers of
+ *                consecutive points. The size of each bin is
+ *                `floor(scoreAndLabels.count() / numBins)`, which means the resulting number
+ *                of bins may not exactly equal numBins. The last bin in each partition may
+ *                be smaller as a result, meaning there may be an extra sample at
+ *                partition boundaries.
  */
 @Experimental
-class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends Logging {
+class BinaryClassificationMetrics(
+    val scoreAndLabels: RDD[(Double, Double)],
+    val numBins: Int) extends Logging {
+
+  require(numBins >= 0, "numBins must be nonnegative")
+
+  /**
+   * Defaults `numBins` to 0.
+   */
+  def this(scoreAndLabels: RDD[(Double, Double)]) = this(scoreAndLabels, 0)
 
   /** Unpersist intermediate RDDs used in the computation. */
   def unpersist() {
@@ -103,7 +124,39 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends
       mergeValue = (c: BinaryLabelCounter, label: Double) => c += label,
       mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2
     ).sortByKey(ascending = false)
-    val agg = counts.values.mapPartitions { iter =>
+
+    val binnedCounts =
+      // Only down-sample if bins is > 0
+      if (numBins == 0) {
+        // Use original directly
+        counts
+      } else {
+        val countsSize = counts.count()
+        // Group the iterator into chunks of about countsSize / numBins points,
+        // so that the resulting number of bins is about numBins
+        var grouping = countsSize / numBins
+        if (grouping < 2) {
+          // numBins was more than half of the size; no real point in down-sampling to bins
+          logInfo(s"Curve is too small ($countsSize) for $numBins bins to be useful")
+          counts
+        } else {
+          if (grouping >= Int.MaxValue) {
+            logWarning(
+              s"Curve too large ($countsSize) for $numBins bins; capping at ${Int.MaxValue}")
+            grouping = Int.MaxValue
+          }
+          counts.mapPartitions(_.grouped(grouping.toInt).map { pairs =>
+            // The score of the combined point will be just the first one's score
+            val firstScore = pairs.head._1
+            // The point will contain all counts in this chunk
+            val agg = new BinaryLabelCounter()
+            pairs.foreach(pair => agg += pair._2)
+            (firstScore, agg)
+          })
+        }
+      }
+
+    val agg = binnedCounts.values.mapPartitions { iter =>
       val agg = new BinaryLabelCounter()
       iter.foreach(agg += _)
       Iterator(agg)
@@ -113,7 +166,7 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends
         (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c)
     val totalCount = partitionwiseCumulativeCounts.last
     logInfo(s"Total counts: $totalCount")
-    val cumulativeCounts = counts.mapPartitionsWithIndex(
+    val cumulativeCounts = binnedCounts.mapPartitionsWithIndex(
       (index: Int, iter: Iterator[(Double, BinaryLabelCounter)]) => {
         val cumCount = partitionwiseCumulativeCounts(index)
         iter.map { case (score, c) =>
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala
index 720bb70b08dbf..3260f27513c7f 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala
@@ -86,20 +86,20 @@ private object IDF {
         df = BDV.zeros(doc.size)
       }
       doc match {
-        case sv: SparseVector =>
-          val nnz = sv.indices.size
+        case SparseVector(size, indices, values) =>
+          val nnz = indices.size
           var k = 0
           while (k < nnz) {
-            if (sv.values(k) > 0) {
-              df(sv.indices(k)) += 1L
+            if (values(k) > 0) {
+              df(indices(k)) += 1L
             }
             k += 1
           }
-        case dv: DenseVector =>
-          val n = dv.size
+        case DenseVector(values) =>
+          val n = values.size
           var j = 0
           while (j < n) {
-            if (dv.values(j) > 0.0) {
+            if (values(j) > 0.0) {
               df(j) += 1L
             }
             j += 1
@@ -174,36 +174,17 @@ class IDFModel private[mllib] (val idf: Vector) extends Serializable {
    */
   def transform(dataset: RDD[Vector]): RDD[Vector] = {
     val bcIdf = dataset.context.broadcast(idf)
-    dataset.mapPartitions { iter =>
-      val thisIdf = bcIdf.value
-      iter.map { v =>
-        val n = v.size
-        v match {
-          case sv: SparseVector =>
-            val nnz = sv.indices.size
-            val newValues = new Array[Double](nnz)
-            var k = 0
-            while (k < nnz) {
-              newValues(k) = sv.values(k) * thisIdf(sv.indices(k))
-              k += 1
-            }
-            Vectors.sparse(n, sv.indices, newValues)
-          case dv: DenseVector =>
-            val newValues = new Array[Double](n)
-            var j = 0
-            while (j < n) {
-              newValues(j) = dv.values(j) * thisIdf(j)
-              j += 1
-            }
-            Vectors.dense(newValues)
-          case other =>
-            throw new UnsupportedOperationException(
-              s"Only sparse and dense vectors are supported but got ${other.getClass}.")
-        }
-      }
-    }
+    dataset.mapPartitions(iter => iter.map(v => IDFModel.transform(bcIdf.value, v)))
   }
 
+  /**
+   * Transforms a term frequency (TF) vector to a TF-IDF vector
+   *
+   * @param v a term frequency vector
+   * @return a TF-IDF vector
+   */
+  def transform(v: Vector): Vector = IDFModel.transform(idf, v)
+
   /**
    * Transforms term frequency (TF) vectors to TF-IDF vectors (Java version).
    * @param dataset a JavaRDD of term frequency vectors
@@ -213,3 +194,39 @@ class IDFModel private[mllib] (val idf: Vector) extends Serializable {
     transform(dataset.rdd).toJavaRDD()
   }
 }
+
+private object IDFModel {
+
+  /**
+   * Transforms a term frequency (TF) vector to a TF-IDF vector with a IDF vector
+   *
+   * @param idf an IDF vector
+   * @param v a term frequence vector
+   * @return a TF-IDF vector
+   */
+  def transform(idf: Vector, v: Vector): Vector = {
+    val n = v.size
+    v match {
+      case SparseVector(size, indices, values) =>
+        val nnz = indices.size
+        val newValues = new Array[Double](nnz)
+        var k = 0
+        while (k < nnz) {
+          newValues(k) = values(k) * idf(indices(k))
+          k += 1
+        }
+        Vectors.sparse(n, indices, newValues)
+      case DenseVector(values) =>
+        val newValues = new Array[Double](n)
+        var j = 0
+        while (j < n) {
+          newValues(j) = values(j) * idf(j)
+          j += 1
+        }
+        Vectors.dense(newValues)
+      case other =>
+        throw new UnsupportedOperationException(
+          s"Only sparse and dense vectors are supported but got ${other.getClass}.")
+    }
+  }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
index 1ced26a9b70a2..32848e039eb81 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
@@ -52,8 +52,8 @@ class Normalizer(p: Double) extends VectorTransformer {
       // However, for sparse vector, the `index` array will not be changed,
       // so we can re-use it to save memory.
       vector match {
-        case dv: DenseVector =>
-          val values = dv.values.clone()
+        case DenseVector(vs) =>
+          val values = vs.clone()
           val size = values.size
           var i = 0
           while (i < size) {
@@ -61,15 +61,15 @@ class Normalizer(p: Double) extends VectorTransformer {
             i += 1
           }
           Vectors.dense(values)
-        case sv: SparseVector =>
-          val values = sv.values.clone()
+        case SparseVector(size, ids, vs) =>
+          val values = vs.clone()
           val nnz = values.size
           var i = 0
           while (i < nnz) {
             values(i) /= norm
             i += 1
           }
-          Vectors.sparse(sv.size, sv.indices, values)
+          Vectors.sparse(size, ids, values)
         case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass)
       }
     } else {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
index 8c4c5db5258d5..3c2091732f9b0 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
@@ -105,8 +105,8 @@ class StandardScalerModel private[mllib] (
       // This can be avoid by having a local reference of `shift`.
       val localShift = shift
       vector match {
-        case dv: DenseVector =>
-          val values = dv.values.clone()
+        case DenseVector(vs) =>
+          val values = vs.clone()
           val size = values.size
           if (withStd) {
             // Having a local reference of `factor` to avoid overhead as the comment before.
@@ -130,8 +130,8 @@ class StandardScalerModel private[mllib] (
       // Having a local reference of `factor` to avoid overhead as the comment before.
       val localFactor = factor
       vector match {
-        case dv: DenseVector =>
-          val values = dv.values.clone()
+        case DenseVector(vs) =>
+          val values = vs.clone()
           val size = values.size
           var i = 0
           while(i < size) {
@@ -139,18 +139,17 @@ class StandardScalerModel private[mllib] (
             i += 1
           }
           Vectors.dense(values)
-        case sv: SparseVector =>
+        case SparseVector(size, indices, vs) =>
           // For sparse vector, the `index` array inside sparse vector object will not be changed,
           // so we can re-use it to save memory.
-          val indices = sv.indices
-          val values = sv.values.clone()
+          val values = vs.clone()
           val nnz = values.size
           var i = 0
           while (i < nnz) {
             values(i) *= localFactor(indices(i))
             i += 1
           }
-          Vectors.sparse(sv.size, indices, values)
+          Vectors.sparse(size, indices, values)
         case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass)
       }
     } else {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
index 7960f3cab576f..d25a7cd5b439d 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
@@ -71,7 +71,8 @@ class Word2Vec extends Serializable with Logging {
   private var numPartitions = 1
   private var numIterations = 1
   private var seed = Utils.random.nextLong()
-
+  private var minCount = 5
+  
   /**
    * Sets vector size (default: 100).
    */
@@ -114,6 +115,15 @@ class Word2Vec extends Serializable with Logging {
     this
   }
 
+  /** 
+   * Sets minCount, the minimum number of times a token must appear to be included in the word2vec 
+   * model's vocabulary (default: 5).
+   */
+  def setMinCount(minCount: Int): this.type = {
+    this.minCount = minCount
+    this
+  }
+  
   private val EXP_TABLE_SIZE = 1000
   private val MAX_EXP = 6
   private val MAX_CODE_LENGTH = 40
@@ -122,9 +132,6 @@ class Word2Vec extends Serializable with Logging {
   /** context words from [-window, window] */
   private val window = 5
 
-  /** minimum frequency to consider a vocabulary word */
-  private val minCount = 5
-
   private var trainWordsCount = 0
   private var vocabSize = 0
   private var vocab: Array[VocabWord] = null
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
index 9fed513becddc..3414daccd7ca4 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
@@ -228,6 +228,32 @@ private[spark] object BLAS extends Serializable with Logging {
     }
     _nativeBLAS
   }
+ 
+  /**
+   * A := alpha * x * x^T^ + A
+   * @param alpha a real scalar that will be multiplied to x * x^T^.
+   * @param x the vector x that contains the n elements.
+   * @param A the symmetric matrix A. Size of n x n.
+   */
+  def syr(alpha: Double, x: DenseVector, A: DenseMatrix) {
+    val mA = A.numRows
+    val nA = A.numCols
+    require(mA == nA, s"A is not a symmetric matrix. A: $mA x $nA")
+    require(mA == x.size, s"The size of x doesn't match the rank of A. A: $mA x $nA, x: ${x.size}")
+
+    nativeBLAS.dsyr("U", x.size, alpha, x.values, 1, A.values, nA)
+
+    // Fill lower triangular part of A
+    var i = 0
+    while (i < mA) {
+      var j = i + 1
+      while (j < nA) {
+        A(j, i) = A(i, j)
+        j += 1
+      }
+      i += 1
+    }    
+  }
 
   /**
    * C := alpha * A * B + beta * C
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
index 327366a1a3a82..5a7281ec6dc3c 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
@@ -17,9 +17,11 @@
 
 package org.apache.spark.mllib.linalg
 
-import java.util.{Random, Arrays}
+import java.util.{Arrays, Random}
 
-import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM}
+import scala.collection.mutable.{ArrayBuilder => MArrayBuilder, HashSet => MHashSet, ArrayBuffer}
+
+import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM}
 
 /**
  * Trait for a local matrix.
@@ -80,6 +82,16 @@ sealed trait Matrix extends Serializable {
 
   /** A human readable representation of the matrix */
   override def toString: String = toBreeze.toString()
+
+  /** Map the values of this matrix using a function. Generates a new matrix. Performs the
+    * function on only the backing array. For example, an operation such as addition or
+    * subtraction will only be performed on the non-zero values in a `SparseMatrix`. */
+  private[mllib] def map(f: Double => Double): Matrix
+
+  /** Update all the values of this matrix using the function f. Performed in-place on the
+    * backing array. For example, an operation such as addition or subtraction will only be
+    * performed on the non-zero values in a `SparseMatrix`. */
+  private[mllib] def update(f: Double => Double): Matrix
 }
 
 /**
@@ -123,6 +135,122 @@ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double])
   }
 
   override def copy = new DenseMatrix(numRows, numCols, values.clone())
+
+  private[mllib] def map(f: Double => Double) = new DenseMatrix(numRows, numCols, values.map(f))
+
+  private[mllib] def update(f: Double => Double): DenseMatrix = {
+    val len = values.length
+    var i = 0
+    while (i < len) {
+      values(i) = f(values(i))
+      i += 1
+    }
+    this
+  }
+
+  /** Generate a `SparseMatrix` from the given `DenseMatrix`. */
+  def toSparse(): SparseMatrix = {
+    val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble
+    val colPtrs: Array[Int] = new Array[Int](numCols + 1)
+    val rowIndices: MArrayBuilder[Int] = new MArrayBuilder.ofInt
+    var nnz = 0
+    var j = 0
+    while (j < numCols) {
+      var i = 0
+      val indStart = j * numRows
+      while (i < numRows) {
+        val v = values(indStart + i)
+        if (v != 0.0) {
+          rowIndices += i
+          spVals += v
+          nnz += 1
+        }
+        i += 1
+      }
+      j += 1
+      colPtrs(j) = nnz
+    }
+    new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), spVals.result())
+  }
+}
+
+/**
+ * Factory methods for [[org.apache.spark.mllib.linalg.DenseMatrix]].
+ */
+object DenseMatrix {
+
+  /**
+   * Generate a `DenseMatrix` consisting of zeros.
+   * @param numRows number of rows of the matrix
+   * @param numCols number of columns of the matrix
+   * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros
+   */
+  def zeros(numRows: Int, numCols: Int): DenseMatrix =
+    new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols))
+
+  /**
+   * Generate a `DenseMatrix` consisting of ones.
+   * @param numRows number of rows of the matrix
+   * @param numCols number of columns of the matrix
+   * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones
+   */
+  def ones(numRows: Int, numCols: Int): DenseMatrix =
+    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0))
+
+  /**
+   * Generate an Identity Matrix in `DenseMatrix` format.
+   * @param n number of rows and columns of the matrix
+   * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal
+   */
+  def eye(n: Int): DenseMatrix = {
+    val identity = DenseMatrix.zeros(n, n)
+    var i = 0
+    while (i < n) {
+      identity.update(i, i, 1.0)
+      i += 1
+    }
+    identity
+  }
+
+  /**
+   * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers.
+   * @param numRows number of rows of the matrix
+   * @param numCols number of columns of the matrix
+   * @param rng a random number generator
+   * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1)
+   */
+  def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = {
+    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble()))
+  }
+
+  /**
+   * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers.
+   * @param numRows number of rows of the matrix
+   * @param numCols number of columns of the matrix
+   * @param rng a random number generator
+   * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1)
+   */
+  def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = {
+    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian()))
+  }
+
+  /**
+   * Generate a diagonal matrix in `DenseMatrix` format from the supplied values.
+   * @param vector a `Vector` that will form the values on the diagonal of the matrix
+   * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values`
+   *         on the diagonal
+   */
+  def diag(vector: Vector): DenseMatrix = {
+    val n = vector.size
+    val matrix = DenseMatrix.zeros(n, n)
+    val values = vector.toArray
+    var i = 0
+    while (i < n) {
+      matrix.update(i, i, values(i))
+      i += 1
+    }
+    matrix
+  }
 }
 
 /**
@@ -156,6 +284,8 @@ class SparseMatrix(
   require(colPtrs.length == numCols + 1, "The length of the column indices should be the " +
     s"number of columns + 1. Currently, colPointers.length: ${colPtrs.length}, " +
     s"numCols: $numCols")
+  require(values.length == colPtrs.last, "The last value of colPtrs must equal the number of " +
+    s"elements. values.length: ${values.length}, colPtrs.last: ${colPtrs.last}")
 
   override def toArray: Array[Double] = {
     val arr = new Array[Double](numRows * numCols)
@@ -188,7 +318,7 @@ class SparseMatrix(
 
   private[mllib] def update(i: Int, j: Int, v: Double): Unit = {
     val ind = index(i, j)
-    if (ind == -1){
+    if (ind == -1) {
       throw new NoSuchElementException("The given row and column indices correspond to a zero " +
         "value. Only non-zero elements in Sparse Matrices can be updated.")
     } else {
@@ -197,6 +327,192 @@ class SparseMatrix(
   }
 
   override def copy = new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone())
+
+  private[mllib] def map(f: Double => Double) =
+    new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.map(f))
+
+  private[mllib] def update(f: Double => Double): SparseMatrix = {
+    val len = values.length
+    var i = 0
+    while (i < len) {
+      values(i) = f(values(i))
+      i += 1
+    }
+    this
+  }
+
+  /** Generate a `DenseMatrix` from the given `SparseMatrix`. */
+  def toDense(): DenseMatrix = {
+    new DenseMatrix(numRows, numCols, toArray)
+  }
+}
+
+/**
+ * Factory methods for [[org.apache.spark.mllib.linalg.SparseMatrix]].
+ */
+object SparseMatrix {
+
+  /**
+   * Generate a `SparseMatrix` from Coordinate List (COO) format. Input must be an array of
+   * (i, j, value) tuples. Entries that have duplicate values of i and j are
+   * added together. Tuples where value is equal to zero will be omitted.
+   * @param numRows number of rows of the matrix
+   * @param numCols number of columns of the matrix
+   * @param entries Array of (i, j, value) tuples
+   * @return The corresponding `SparseMatrix`
+   */
+  def fromCOO(numRows: Int, numCols: Int, entries: Iterable[(Int, Int, Double)]): SparseMatrix = {
+    val sortedEntries = entries.toSeq.sortBy(v => (v._2, v._1))
+    val numEntries = sortedEntries.size
+    if (sortedEntries.nonEmpty) {
+      // Since the entries are sorted by column index, we only need to check the first and the last.
+      for (col <- Seq(sortedEntries.head._2, sortedEntries.last._2)) {
+        require(col >= 0 && col < numCols, s"Column index out of range [0, $numCols): $col.")
+      }
+    }
+    val colPtrs = new Array[Int](numCols + 1)
+    val rowIndices = MArrayBuilder.make[Int]
+    rowIndices.sizeHint(numEntries)
+    val values = MArrayBuilder.make[Double]
+    values.sizeHint(numEntries)
+    var nnz = 0
+    var prevCol = 0
+    var prevRow = -1
+    var prevVal = 0.0
+    // Append a dummy entry to include the last one at the end of the loop.
+    (sortedEntries.view :+ (numRows, numCols, 1.0)).foreach { case (i, j, v) =>
+      if (v != 0) {
+        if (i == prevRow && j == prevCol) {
+          prevVal += v
+        } else {
+          if (prevVal != 0) {
+            require(prevRow >= 0 && prevRow < numRows,
+              s"Row index out of range [0, $numRows): $prevRow.")
+            nnz += 1
+            rowIndices += prevRow
+            values += prevVal
+          }
+          prevRow = i
+          prevVal = v
+          while (prevCol < j) {
+            colPtrs(prevCol + 1) = nnz
+            prevCol += 1
+          }
+        }
+      }
+    }
+    new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), values.result())
+  }
+
+  /**
+   * Generate an Identity Matrix in `SparseMatrix` format.
+   * @param n number of rows and columns of the matrix
+   * @return `SparseMatrix` with size `n` x `n` and values of ones on the diagonal
+   */
+  def speye(n: Int): SparseMatrix = {
+    new SparseMatrix(n, n, (0 to n).toArray, (0 until n).toArray, Array.fill(n)(1.0))
+  }
+
+  /**
+   * Generates the skeleton of a random `SparseMatrix` with a given random number generator.
+   * The values of the matrix returned are undefined.
+   */
+  private def genRandMatrix(
+      numRows: Int,
+      numCols: Int,
+      density: Double,
+      rng: Random): SparseMatrix = {
+    require(numRows > 0, s"numRows must be greater than 0 but got $numRows")
+    require(numCols > 0, s"numCols must be greater than 0 but got $numCols")
+    require(density >= 0.0 && density <= 1.0,
+      s"density must be a double in the range 0.0 <= d <= 1.0. Currently, density: $density")
+    val size = numRows.toLong * numCols
+    val expected = size * density
+    assert(expected < Int.MaxValue,
+      "The expected number of nonzeros cannot be greater than Int.MaxValue.")
+    val nnz = math.ceil(expected).toInt
+    if (density == 0.0) {
+      new SparseMatrix(numRows, numCols, new Array[Int](numCols + 1), Array[Int](), Array[Double]())
+    } else if (density == 1.0) {
+      val colPtrs = Array.tabulate(numCols + 1)(j => j * numRows)
+      val rowIndices = Array.tabulate(size.toInt)(idx => idx % numRows)
+      new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](numRows * numCols))
+    } else if (density < 0.34) {
+      // draw-by-draw, expected number of iterations is less than 1.5 * nnz
+      val entries = MHashSet[(Int, Int)]()
+      while (entries.size < nnz) {
+        entries += ((rng.nextInt(numRows), rng.nextInt(numCols)))
+      }
+      SparseMatrix.fromCOO(numRows, numCols, entries.map(v => (v._1, v._2, 1.0)))
+    } else {
+      // selection-rejection method
+      var idx = 0L
+      var numSelected = 0
+      var j = 0
+      val colPtrs = new Array[Int](numCols + 1)
+      val rowIndices = new Array[Int](nnz)
+      while (j < numCols && numSelected < nnz) {
+        var i = 0
+        while (i < numRows && numSelected < nnz) {
+          if (rng.nextDouble() < 1.0 * (nnz - numSelected) / (size - idx)) {
+            rowIndices(numSelected) = i
+            numSelected += 1
+          }
+          i += 1
+          idx += 1
+        }
+        colPtrs(j + 1) = numSelected
+        j += 1
+      }
+      new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](nnz))
+    }
+  }
+
+  /**
+   * Generate a `SparseMatrix` consisting of i.i.d. uniform random numbers. The number of non-zero
+   * elements equal the ceiling of `numRows` x `numCols` x `density`
+   *
+   * @param numRows number of rows of the matrix
+   * @param numCols number of columns of the matrix
+   * @param density the desired density for the matrix
+   * @param rng a random number generator
+   * @return `SparseMatrix` with size `numRows` x `numCols` and values in U(0, 1)
+   */
+  def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = {
+    val mat = genRandMatrix(numRows, numCols, density, rng)
+    mat.update(i => rng.nextDouble())
+  }
+
+  /**
+   * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers.
+   * @param numRows number of rows of the matrix
+   * @param numCols number of columns of the matrix
+   * @param density the desired density for the matrix
+   * @param rng a random number generator
+   * @return `SparseMatrix` with size `numRows` x `numCols` and values in N(0, 1)
+   */
+  def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = {
+    val mat = genRandMatrix(numRows, numCols, density, rng)
+    mat.update(i => rng.nextGaussian())
+  }
+
+  /**
+   * Generate a diagonal matrix in `SparseMatrix` format from the supplied values.
+   * @param vector a `Vector` that will form the values on the diagonal of the matrix
+   * @return Square `SparseMatrix` with size `values.length` x `values.length` and non-zero
+   *         `values` on the diagonal
+   */
+  def diag(vector: Vector): SparseMatrix = {
+    val n = vector.size
+    vector match {
+      case sVec: SparseVector =>
+        SparseMatrix.fromCOO(n, n, sVec.indices.zip(sVec.values).map(v => (v._1, v._1, v._2)))
+      case dVec: DenseVector =>
+        val entries = dVec.values.zipWithIndex
+        val nnzVals = entries.filter(v => v._1 != 0.0)
+        SparseMatrix.fromCOO(n, n, nnzVals.map(v => (v._2, v._2, v._1)))
+    }
+  }
 }
 
 /**
@@ -256,72 +572,250 @@ object Matrices {
    * Generate a `DenseMatrix` consisting of zeros.
    * @param numRows number of rows of the matrix
    * @param numCols number of columns of the matrix
-   * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros
+   * @return `Matrix` with size `numRows` x `numCols` and values of zeros
    */
-  def zeros(numRows: Int, numCols: Int): Matrix =
-    new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols))
+  def zeros(numRows: Int, numCols: Int): Matrix = DenseMatrix.zeros(numRows, numCols)
 
   /**
    * Generate a `DenseMatrix` consisting of ones.
    * @param numRows number of rows of the matrix
    * @param numCols number of columns of the matrix
-   * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones
+   * @return `Matrix` with size `numRows` x `numCols` and values of ones
    */
-  def ones(numRows: Int, numCols: Int): Matrix =
-    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0))
+  def ones(numRows: Int, numCols: Int): Matrix = DenseMatrix.ones(numRows, numCols)
 
   /**
-   * Generate an Identity Matrix in `DenseMatrix` format.
+   * Generate a dense Identity Matrix in `Matrix` format.
    * @param n number of rows and columns of the matrix
-   * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal
+   * @return `Matrix` with size `n` x `n` and values of ones on the diagonal
    */
-  def eye(n: Int): Matrix = {
-    val identity = Matrices.zeros(n, n)
-    var i = 0
-    while (i < n){
-      identity.update(i, i, 1.0)
-      i += 1
-    }
-    identity
-  }
+  def eye(n: Int): Matrix = DenseMatrix.eye(n)
+
+  /**
+   * Generate a sparse Identity Matrix in `Matrix` format.
+   * @param n number of rows and columns of the matrix
+   * @return `Matrix` with size `n` x `n` and values of ones on the diagonal
+   */
+  def speye(n: Int): Matrix = SparseMatrix.speye(n)
 
   /**
    * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers.
    * @param numRows number of rows of the matrix
    * @param numCols number of columns of the matrix
    * @param rng a random number generator
-   * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1)
+   * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1)
    */
-  def rand(numRows: Int, numCols: Int, rng: Random): Matrix = {
-    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble()))
-  }
+  def rand(numRows: Int, numCols: Int, rng: Random): Matrix =
+    DenseMatrix.rand(numRows, numCols, rng)
+
+  /**
+   * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers.
+   * @param numRows number of rows of the matrix
+   * @param numCols number of columns of the matrix
+   * @param density the desired density for the matrix
+   * @param rng a random number generator
+   * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1)
+   */
+  def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix =
+    SparseMatrix.sprand(numRows, numCols, density, rng)
 
   /**
    * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers.
    * @param numRows number of rows of the matrix
    * @param numCols number of columns of the matrix
    * @param rng a random number generator
-   * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1)
+   * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1)
    */
-  def randn(numRows: Int, numCols: Int, rng: Random): Matrix = {
-    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian()))
-  }
+  def randn(numRows: Int, numCols: Int, rng: Random): Matrix =
+    DenseMatrix.randn(numRows, numCols, rng)
+
+  /**
+   * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers.
+   * @param numRows number of rows of the matrix
+   * @param numCols number of columns of the matrix
+   * @param density the desired density for the matrix
+   * @param rng a random number generator
+   * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1)
+   */
+  def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix =
+    SparseMatrix.sprandn(numRows, numCols, density, rng)
 
   /**
    * Generate a diagonal matrix in `DenseMatrix` format from the supplied values.
    * @param vector a `Vector` tat will form the values on the diagonal of the matrix
-   * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values`
+   * @return Square `Matrix` with size `values.length` x `values.length` and `values`
    *         on the diagonal
    */
-  def diag(vector: Vector): Matrix = {
-    val n = vector.size
-    val matrix = Matrices.eye(n)
-    val values = vector.toArray
-    var i = 0
-    while (i < n) {
-      matrix.update(i, i, values(i))
-      i += 1
+  def diag(vector: Vector): Matrix = DenseMatrix.diag(vector)
+
+  /**
+   * Horizontally concatenate a sequence of matrices. The returned matrix will be in the format
+   * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in
+   * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned.
+   * @param matrices array of matrices
+   * @return a single `Matrix` composed of the matrices that were horizontally concatenated
+   */
+  def horzcat(matrices: Array[Matrix]): Matrix = {
+    if (matrices.isEmpty) {
+      return new DenseMatrix(0, 0, Array[Double]())
+    } else if (matrices.size == 1) {
+      return matrices(0)
+    }
+    val numRows = matrices(0).numRows
+    var hasSparse = false
+    var numCols = 0
+    matrices.foreach { mat =>
+      require(numRows == mat.numRows, "The number of rows of the matrices in this sequence, " +
+        "don't match!")
+      mat match {
+        case sparse: SparseMatrix => hasSparse = true
+        case dense: DenseMatrix => // empty on purpose
+        case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " +
+          s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}")
+      }
+      numCols += mat.numCols
+    }
+    if (!hasSparse) {
+      new DenseMatrix(numRows, numCols, matrices.flatMap(_.toArray))
+    } else {
+      var startCol = 0
+      val entries: Array[(Int, Int, Double)] = matrices.flatMap {
+        case spMat: SparseMatrix =>
+          var j = 0
+          val colPtrs = spMat.colPtrs
+          val rowIndices = spMat.rowIndices
+          val values = spMat.values
+          val data = new Array[(Int, Int, Double)](values.length)
+          val nCols = spMat.numCols
+          while (j < nCols) {
+            var idx = colPtrs(j)
+            while (idx < colPtrs(j + 1)) {
+              val i = rowIndices(idx)
+              val v = values(idx)
+              data(idx) = (i, j + startCol, v)
+              idx += 1
+            }
+            j += 1
+          }
+          startCol += nCols
+          data
+        case dnMat: DenseMatrix =>
+          val data = new ArrayBuffer[(Int, Int, Double)]()
+          var j = 0
+          val nCols = dnMat.numCols
+          val nRows = dnMat.numRows
+          val values = dnMat.values
+          while (j < nCols) {
+            var i = 0
+            val indStart = j * nRows
+            while (i < nRows) {
+              val v = values(indStart + i)
+              if (v != 0.0) {
+                data.append((i, j + startCol, v))
+              }
+              i += 1
+            }
+            j += 1
+          }
+          startCol += nCols
+          data
+      }
+      SparseMatrix.fromCOO(numRows, numCols, entries)
+    }
+  }
+
+  /**
+   * Vertically concatenate a sequence of matrices. The returned matrix will be in the format
+   * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in
+   * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned.
+   * @param matrices array of matrices
+   * @return a single `Matrix` composed of the matrices that were vertically concatenated
+   */
+  def vertcat(matrices: Array[Matrix]): Matrix = {
+    if (matrices.isEmpty) {
+      return new DenseMatrix(0, 0, Array[Double]())
+    } else if (matrices.size == 1) {
+      return matrices(0)
+    }
+    val numCols = matrices(0).numCols
+    var hasSparse = false
+    var numRows = 0
+    matrices.foreach { mat =>
+      require(numCols == mat.numCols, "The number of rows of the matrices in this sequence, " +
+        "don't match!")
+      mat match {
+        case sparse: SparseMatrix =>
+          hasSparse = true
+        case dense: DenseMatrix =>
+        case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " +
+          s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}")
+      }
+      numRows += mat.numRows
+
+    }
+    if (!hasSparse) {
+      val allValues = new Array[Double](numRows * numCols)
+      var startRow = 0
+      matrices.foreach { mat =>
+        var j = 0
+        val nRows = mat.numRows
+        val values = mat.toArray
+        while (j < numCols) {
+          var i = 0
+          val indStart = j * numRows + startRow
+          val subMatStart = j * nRows
+          while (i < nRows) {
+            allValues(indStart + i) = values(subMatStart + i)
+            i += 1
+          }
+          j += 1
+        }
+        startRow += nRows
+      }
+      new DenseMatrix(numRows, numCols, allValues)
+    } else {
+      var startRow = 0
+      val entries: Array[(Int, Int, Double)] = matrices.flatMap {
+        case spMat: SparseMatrix =>
+          var j = 0
+          val colPtrs = spMat.colPtrs
+          val rowIndices = spMat.rowIndices
+          val values = spMat.values
+          val data = new Array[(Int, Int, Double)](values.length)
+          while (j < numCols) {
+            var idx = colPtrs(j)
+            while (idx < colPtrs(j + 1)) {
+              val i = rowIndices(idx)
+              val v = values(idx)
+              data(idx) = (i + startRow, j, v)
+              idx += 1
+            }
+            j += 1
+          }
+          startRow += spMat.numRows
+          data
+        case dnMat: DenseMatrix =>
+          val data = new ArrayBuffer[(Int, Int, Double)]()
+          var j = 0
+          val nCols = dnMat.numCols
+          val nRows = dnMat.numRows
+          val values = dnMat.values
+          while (j < nCols) {
+            var i = 0
+            val indStart = j * nRows
+            while (i < nRows) {
+              val v = values(indStart + i)
+              if (v != 0.0) {
+                data.append((i + startRow, j, v))
+              }
+              i += 1
+            }
+            j += 1
+          }
+          startRow += nRows
+          data
+      }
+      SparseMatrix.fromCOO(numRows, numCols, entries)
     }
-    matrix
   }
 }
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
index 47d1a76fa361d..bf1faa25ef0e0 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
@@ -108,16 +108,16 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] {
   override def serialize(obj: Any): Row = {
     val row = new GenericMutableRow(4)
     obj match {
-      case sv: SparseVector =>
+      case SparseVector(size, indices, values) =>
         row.setByte(0, 0)
-        row.setInt(1, sv.size)
-        row.update(2, sv.indices.toSeq)
-        row.update(3, sv.values.toSeq)
-      case dv: DenseVector =>
+        row.setInt(1, size)
+        row.update(2, indices.toSeq)
+        row.update(3, values.toSeq)
+      case DenseVector(values) =>
         row.setByte(0, 1)
         row.setNullAt(1)
         row.setNullAt(2)
-        row.update(3, dv.values.toSeq)
+        row.update(3, values.toSeq)
     }
     row
   }
@@ -268,11 +268,11 @@ object Vectors {
    * @param p norm.
    * @return norm in L^p^ space.
    */
-  private[spark] def norm(vector: Vector, p: Double): Double = {
+  def norm(vector: Vector, p: Double): Double = {
     require(p >= 1.0)
     val values = vector match {
-      case dv: DenseVector => dv.values
-      case sv: SparseVector => sv.values
+      case DenseVector(vs) => vs
+      case SparseVector(n, ids, vs) => vs
       case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass)
     }
     val size = values.size
@@ -312,6 +312,87 @@ object Vectors {
       math.pow(sum, 1.0 / p)
     }
   }
+ 
+  /**
+   * Returns the squared distance between two Vectors.
+   * @param v1 first Vector.
+   * @param v2 second Vector.
+   * @return squared distance between two Vectors.
+   */
+  def sqdist(v1: Vector, v2: Vector): Double = {
+    var squaredDistance = 0.0
+    (v1, v2) match { 
+      case (v1: SparseVector, v2: SparseVector) =>
+        val v1Values = v1.values
+        val v1Indices = v1.indices
+        val v2Values = v2.values
+        val v2Indices = v2.indices
+        val nnzv1 = v1Indices.size
+        val nnzv2 = v2Indices.size
+        
+        var kv1 = 0
+        var kv2 = 0
+        while (kv1 < nnzv1 || kv2 < nnzv2) {
+          var score = 0.0
+ 
+          if (kv2 >= nnzv2 || (kv1 < nnzv1 && v1Indices(kv1) < v2Indices(kv2))) {
+            score = v1Values(kv1)
+            kv1 += 1
+          } else if (kv1 >= nnzv1 || (kv2 < nnzv2 && v2Indices(kv2) < v1Indices(kv1))) {
+            score = v2Values(kv2)
+            kv2 += 1
+          } else {
+            score = v1Values(kv1) - v2Values(kv2)
+            kv1 += 1
+            kv2 += 1
+          }
+          squaredDistance += score * score
+        }
+
+      case (v1: SparseVector, v2: DenseVector) if v1.indices.length / v1.size < 0.5 =>
+        squaredDistance = sqdist(v1, v2)
+
+      case (v1: DenseVector, v2: SparseVector) if v2.indices.length / v2.size < 0.5 =>
+        squaredDistance = sqdist(v2, v1)
+
+      // When a SparseVector is approximately dense, we treat it as a DenseVector
+      case (v1, v2) =>
+        squaredDistance = v1.toArray.zip(v2.toArray).foldLeft(0.0){ (distance, elems) =>
+          val score = elems._1 - elems._2
+          distance + score * score
+        }
+    }
+    squaredDistance
+  }
+
+  /**
+   * Returns the squared distance between DenseVector and SparseVector.
+   */
+  private[mllib] def sqdist(v1: SparseVector, v2: DenseVector): Double = {
+    var kv1 = 0
+    var kv2 = 0
+    val indices = v1.indices
+    var squaredDistance = 0.0
+    val nnzv1 = indices.size
+    val nnzv2 = v2.size
+    var iv1 = if (nnzv1 > 0) indices(kv1) else -1
+   
+    while (kv2 < nnzv2) {
+      var score = 0.0
+      if (kv2 != iv1) {
+        score = v2(kv2)
+      } else {
+        score = v1.values(kv1) - v2(kv2)
+        if (kv1 < nnzv1 - 1) {
+          kv1 += 1
+          iv1 = indices(kv1)
+        }
+      }
+      squaredDistance += score * score
+      kv2 += 1
+    }
+    squaredDistance
+  }
 }
 
 /**
@@ -346,6 +427,10 @@ class DenseVector(val values: Array[Double]) extends Vector {
   }
 }
 
+object DenseVector {
+  def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values)
+}
+
 /**
  * A sparse vector represented by an index array and an value array.
  *
@@ -393,3 +478,8 @@ class SparseVector(
     }
   }
 }
+
+object SparseVector {
+  def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] =
+    Some((sv.size, sv.indices, sv.values))
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
index 5c1acca0ec532..36d8cadd2bdd7 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
@@ -142,7 +142,7 @@ class IndexedRowMatrix(
     val mat = BDM.zeros[Double](m, n)
     rows.collect().foreach { case IndexedRow(rowIndex, vector) =>
       val i = rowIndex.toInt
-      vector.toBreeze.activeIterator.foreach { case (j, v) =>
+      vector.foreachActive { case (j, v) =>
         mat(i, j) = v
       }
     }
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala
index 10a515af88802..fbd35e372f9b1 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala
@@ -528,21 +528,21 @@ class RowMatrix(
       iter.flatMap { row =>
         val buf = new ListBuffer[((Int, Int), Double)]()
         row match {
-          case sv: SparseVector =>
-            val nnz = sv.indices.size
+          case SparseVector(size, indices, values) =>
+            val nnz = indices.size
             var k = 0
             while (k < nnz) {
-              scaled(k) = sv.values(k) / q(sv.indices(k))
+              scaled(k) = values(k) / q(indices(k))
               k += 1
             }
             k = 0
             while (k < nnz) {
-              val i = sv.indices(k)
+              val i = indices(k)
               val iVal = scaled(k)
               if (iVal != 0 && rand.nextDouble() < p(i)) {
                 var l = k + 1
                 while (l < nnz) {
-                  val j = sv.indices(l)
+                  val j = indices(l)
                   val jVal = scaled(l)
                   if (jVal != 0 && rand.nextDouble() < p(j)) {
                     buf += (((i, j), iVal * jVal))
@@ -552,11 +552,11 @@ class RowMatrix(
               }
               k += 1
             }
-          case dv: DenseVector =>
-            val n = dv.values.size
+          case DenseVector(values) =>
+            val n = values.size
             var i = 0
             while (i < n) {
-              scaled(i) = dv.values(i) / q(i)
+              scaled(i) = values(i) / q(i)
               i += 1
             }
             i = 0
@@ -588,8 +588,8 @@ class RowMatrix(
     val n = numCols().toInt
     val mat = BDM.zeros[Double](m, n)
     var i = 0
-    rows.collect().foreach { v =>
-      v.toBreeze.activeIterator.foreach { case (j, v) =>
+    rows.collect().foreach { vector =>
+      vector.foreachActive { case (j, v) =>
         mat(i, j) = v
       }
       i += 1
@@ -620,11 +620,9 @@ object RowMatrix {
     // TODO: Find a better home (breeze?) for this method.
     val n = v.size
     v match {
-      case dv: DenseVector =>
-        blas.dspr("U", n, alpha, dv.values, 1, U)
-      case sv: SparseVector =>
-        val indices = sv.indices
-        val values = sv.values
+      case DenseVector(values) =>
+        blas.dspr("U", n, alpha, values, 1, U)
+      case SparseVector(size, indices, values) =>
         val nnz = indices.length
         var colStartIdx = 0
         var prevCol = 0
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
index 45dbf6044fcc5..1ca0f36c6ac34 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
@@ -20,6 +20,7 @@ package org.apache.spark.mllib.optimization
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.mllib.linalg.{Vector, Vectors}
 import org.apache.spark.mllib.linalg.BLAS.{axpy, dot, scal}
+import org.apache.spark.mllib.util.MLUtils
 
 /**
  * :: DeveloperApi ::
@@ -66,9 +67,10 @@ class LogisticGradient extends Gradient {
     scal(gradientMultiplier, gradient)
     val loss =
       if (label > 0) {
-        math.log1p(math.exp(margin)) // log1p is log(1+p) but more accurate for small p
+        // The following is equivalent to log(1 + exp(margin)) but more numerically stable.
+        MLUtils.log1pExp(margin)
       } else {
-        math.log1p(math.exp(margin)) - margin
+        MLUtils.log1pExp(margin) - margin
       }
 
     (gradient, loss)
@@ -83,9 +85,10 @@ class LogisticGradient extends Gradient {
     val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label
     axpy(gradientMultiplier, data, cumGradient)
     if (label > 0) {
-      math.log1p(math.exp(margin))
+      // The following is equivalent to log(1 + exp(margin)) but more numerically stable.
+      MLUtils.log1pExp(margin)
     } else {
-      math.log1p(math.exp(margin)) - margin
+      MLUtils.log1pExp(margin) - margin
     }
   }
 }
@@ -94,16 +97,16 @@ class LogisticGradient extends Gradient {
  * :: DeveloperApi ::
  * Compute gradient and loss for a Least-squared loss function, as used in linear regression.
  * This is correct for the averaged least squares loss function (mean squared error)
- *              L = 1/n ||A weights-y||^2
+ *              L = 1/2n ||A weights-y||^2
  * See also the documentation for the precise formulation.
  */
 @DeveloperApi
 class LeastSquaresGradient extends Gradient {
   override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = {
     val diff = dot(data, weights) - label
-    val loss = diff * diff
+    val loss = diff * diff / 2.0
     val gradient = data.copy
-    scal(2.0 * diff, gradient)
+    scal(diff, gradient)
     (gradient, loss)
   }
 
@@ -113,8 +116,8 @@ class LeastSquaresGradient extends Gradient {
       weights: Vector,
       cumGradient: Vector): Double = {
     val diff = dot(data, weights) - label
-    axpy(2.0 * diff, data, cumGradient)
-    diff * diff
+    axpy(diff, data, cumGradient)
+    diff * diff / 2.0
   }
 }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala
index 51f9b8657c640..405bae62ee8b6 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala
@@ -17,7 +17,8 @@
 
 package org.apache.spark.mllib.random
 
-import org.apache.commons.math3.distribution.PoissonDistribution
+import org.apache.commons.math3.distribution.{ExponentialDistribution,
+  GammaDistribution, LogNormalDistribution, PoissonDistribution}
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom}
@@ -88,14 +89,76 @@ class StandardNormalGenerator extends RandomDataGenerator[Double] {
 @DeveloperApi
 class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] {
 
-  private var rng = new PoissonDistribution(mean)
+  private val rng = new PoissonDistribution(mean)
 
   override def nextValue(): Double = rng.sample()
 
   override def setSeed(seed: Long) {
-    rng = new PoissonDistribution(mean)
     rng.reseedRandomGenerator(seed)
   }
 
   override def copy(): PoissonGenerator = new PoissonGenerator(mean)
 }
+
+/**
+ * :: DeveloperApi ::
+ * Generates i.i.d. samples from the exponential distribution with the given mean.
+ *
+ * @param mean mean for the exponential distribution.
+ */
+@DeveloperApi
+class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] {
+
+  private val rng = new ExponentialDistribution(mean)
+
+  override def nextValue(): Double = rng.sample()
+
+  override def setSeed(seed: Long) {
+    rng.reseedRandomGenerator(seed)
+  }
+
+  override def copy(): ExponentialGenerator = new ExponentialGenerator(mean)
+}
+
+/**
+ * :: DeveloperApi ::
+ * Generates i.i.d. samples from the gamma distribution with the given shape and scale.
+ *
+ * @param shape shape for the gamma distribution.
+ * @param scale scale for the gamma distribution
+ */
+@DeveloperApi
+class GammaGenerator(val shape: Double, val scale: Double) extends RandomDataGenerator[Double] {
+
+  private val rng = new GammaDistribution(shape, scale)
+
+  override def nextValue(): Double = rng.sample()
+
+  override def setSeed(seed: Long) {
+    rng.reseedRandomGenerator(seed)
+  }
+
+  override def copy(): GammaGenerator = new GammaGenerator(shape, scale)
+}
+
+/**
+ * :: DeveloperApi ::
+ * Generates i.i.d. samples from the log normal distribution with the
+ * given mean and standard deviation.
+ *
+ * @param mean mean for the log normal distribution.
+ * @param std standard deviation for the log normal distribution
+ */
+@DeveloperApi
+class LogNormalGenerator(val mean: Double, val std: Double) extends RandomDataGenerator[Double] {
+
+  private val rng = new LogNormalDistribution(mean, std)
+
+  override def nextValue(): Double = rng.sample()
+
+  override def setSeed(seed: Long) {
+    rng.reseedRandomGenerator(seed)
+  }
+
+  override def copy(): LogNormalGenerator = new LogNormalGenerator(mean, std)
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala
index c5f4b084321f7..955c593a085d5 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala
@@ -176,6 +176,176 @@ object RandomRDDs {
     JavaDoubleRDD.fromRDD(poissonRDD(jsc.sc, mean, size))
   }
 
+  /**
+   * Generates an RDD comprised of i.i.d. samples from the exponential distribution with
+   * the input mean.
+   *
+   * @param sc SparkContext used to create the RDD.
+   * @param mean Mean, or 1 / lambda, for the exponential distribution.
+   * @param size Size of the RDD.
+   * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`).
+   * @param seed Random seed (default: a random long integer).
+   * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean).
+   */
+  def exponentialRDD(
+      sc: SparkContext,
+      mean: Double,
+      size: Long,
+      numPartitions: Int = 0,
+      seed: Long = Utils.random.nextLong()): RDD[Double] = {
+    val exponential = new ExponentialGenerator(mean)
+    randomRDD(sc, exponential, size, numPartitionsOrDefault(sc, numPartitions), seed)
+  }
+
+  /**
+   * Java-friendly version of [[RandomRDDs#exponentialRDD]].
+   */
+  def exponentialJavaRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      size: Long,
+      numPartitions: Int,
+      seed: Long): JavaDoubleRDD = {
+    JavaDoubleRDD.fromRDD(exponentialRDD(jsc.sc, mean, size, numPartitions, seed))
+  }
+
+  /**
+   * [[RandomRDDs#exponentialJavaRDD]] with the default seed.
+   */
+  def exponentialJavaRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      size: Long,
+      numPartitions: Int): JavaDoubleRDD = {
+    JavaDoubleRDD.fromRDD(exponentialRDD(jsc.sc, mean, size, numPartitions))
+  }
+
+  /**
+   * [[RandomRDDs#exponentialJavaRDD]] with the default number of partitions and the default seed.
+   */
+  def exponentialJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = {
+    JavaDoubleRDD.fromRDD(exponentialRDD(jsc.sc, mean, size))
+  }
+
+  /**
+   * Generates an RDD comprised of i.i.d. samples from the gamma distribution with the input
+   *  shape and scale.
+   *
+   * @param sc SparkContext used to create the RDD.
+   * @param shape shape parameter (> 0) for the gamma distribution
+   * @param scale scale parameter (> 0) for the gamma distribution  
+   * @param size Size of the RDD.
+   * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`).
+   * @param seed Random seed (default: a random long integer).
+   * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean).
+   */
+  def gammaRDD(
+      sc: SparkContext,
+      shape: Double,
+      scale: Double,
+      size: Long,
+      numPartitions: Int = 0,
+      seed: Long = Utils.random.nextLong()): RDD[Double] = {
+    val gamma = new GammaGenerator(shape, scale)
+    randomRDD(sc, gamma, size, numPartitionsOrDefault(sc, numPartitions), seed)
+  }
+
+  /**
+   * Java-friendly version of [[RandomRDDs#gammaRDD]].
+   */
+  def gammaJavaRDD(
+      jsc: JavaSparkContext,
+      shape: Double,
+      scale: Double,
+      size: Long,
+      numPartitions: Int,
+      seed: Long): JavaDoubleRDD = {
+    JavaDoubleRDD.fromRDD(gammaRDD(jsc.sc, shape, scale, size, numPartitions, seed))
+  }
+
+  /**
+   * [[RandomRDDs#gammaJavaRDD]] with the default seed.
+   */
+  def gammaJavaRDD(
+      jsc: JavaSparkContext,
+      shape: Double,
+      scale: Double,
+      size: Long,
+      numPartitions: Int): JavaDoubleRDD = {
+    JavaDoubleRDD.fromRDD(gammaRDD(jsc.sc, shape, scale, size, numPartitions))
+  }
+
+  /**
+   * [[RandomRDDs#gammaJavaRDD]] with the default number of partitions and the default seed.
+   */
+  def gammaJavaRDD(
+    jsc: JavaSparkContext,
+    shape: Double,
+    scale: Double,
+    size: Long): JavaDoubleRDD = {
+    JavaDoubleRDD.fromRDD(gammaRDD(jsc.sc, shape, scale, size))
+  }
+
+  /**
+   * Generates an RDD comprised of i.i.d. samples from the log normal distribution with the input
+   *  mean and standard deviation
+   *
+   * @param sc SparkContext used to create the RDD.
+   * @param mean mean for the log normal distribution
+   * @param std standard deviation for the log normal distribution  
+   * @param size Size of the RDD.
+   * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`).
+   * @param seed Random seed (default: a random long integer).
+   * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean).
+   */
+  def logNormalRDD(
+      sc: SparkContext,
+      mean: Double,
+      std: Double,
+      size: Long,
+      numPartitions: Int = 0,
+      seed: Long = Utils.random.nextLong()): RDD[Double] = {
+    val logNormal = new LogNormalGenerator(mean, std)
+    randomRDD(sc, logNormal, size, numPartitionsOrDefault(sc, numPartitions), seed)
+  }
+
+  /**
+   * Java-friendly version of [[RandomRDDs#logNormalRDD]].
+   */
+  def logNormalJavaRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      std: Double,
+      size: Long,
+      numPartitions: Int,
+      seed: Long): JavaDoubleRDD = {
+    JavaDoubleRDD.fromRDD(logNormalRDD(jsc.sc, mean, std, size, numPartitions, seed))
+  }
+
+  /**
+   * [[RandomRDDs#logNormalJavaRDD]] with the default seed.
+   */
+  def logNormalJavaRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      std: Double,
+      size: Long,
+      numPartitions: Int): JavaDoubleRDD = {
+    JavaDoubleRDD.fromRDD(logNormalRDD(jsc.sc, mean, std, size, numPartitions))
+  }
+
+  /**
+   * [[RandomRDDs#logNormalJavaRDD]] with the default number of partitions and the default seed.
+   */
+  def logNormalJavaRDD(
+    jsc: JavaSparkContext,
+    mean: Double,
+    std: Double,
+    size: Long): JavaDoubleRDD = {
+    JavaDoubleRDD.fromRDD(logNormalRDD(jsc.sc, mean, std, size))
+  }
+
+
   /**
    * :: DeveloperApi ::
    * Generates an RDD comprised of i.i.d. samples produced by the input RandomDataGenerator.
@@ -307,6 +477,72 @@ object RandomRDDs {
     normalVectorRDD(jsc.sc, numRows, numCols).toJavaRDD()
   }
 
+  /**
+   * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from a
+   * log normal distribution.
+   *
+   * @param sc SparkContext used to create the RDD.
+   * @param mean Mean of the log normal distribution.
+   * @param std Standard deviation of the log normal distribution.
+   * @param numRows Number of Vectors in the RDD.
+   * @param numCols Number of elements in each Vector.
+   * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`).
+   * @param seed Random seed (default: a random long integer).
+   * @return RDD[Vector] with vectors containing i.i.d. samples.
+   */
+  def logNormalVectorRDD(
+      sc: SparkContext,
+      mean: Double,
+      std: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: Int = 0,
+      seed: Long = Utils.random.nextLong()): RDD[Vector] = {
+    val logNormal = new LogNormalGenerator(mean, std)
+    randomVectorRDD(sc, logNormal, numRows, numCols,
+      numPartitionsOrDefault(sc, numPartitions), seed)
+  }
+
+  /**
+   * Java-friendly version of [[RandomRDDs#logNormalVectorRDD]].
+   */
+  def logNormalJavaVectorRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      std: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: Int,
+      seed: Long): JavaRDD[Vector] = {
+    logNormalVectorRDD(jsc.sc, mean, std, numRows, numCols, numPartitions, seed).toJavaRDD()
+  }
+
+  /**
+   * [[RandomRDDs#logNormalJavaVectorRDD]] with the default seed.
+   */
+  def logNormalJavaVectorRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      std: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: Int): JavaRDD[Vector] = {
+    logNormalVectorRDD(jsc.sc, mean, std, numRows, numCols, numPartitions).toJavaRDD()
+  }
+
+  /**
+   * [[RandomRDDs#logNormalJavaVectorRDD]] with the default number of partitions and
+   * the default seed.
+   */
+  def logNormalJavaVectorRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      std: Double,
+      numRows: Long,
+      numCols: Int): JavaRDD[Vector] = {
+    logNormalVectorRDD(jsc.sc, mean, std, numRows, numCols).toJavaRDD()
+  }
+
   /**
    * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the
    * Poisson distribution with the input mean.
@@ -366,6 +602,133 @@ object RandomRDDs {
     poissonVectorRDD(jsc.sc, mean, numRows, numCols).toJavaRDD()
   }
 
+  /**
+   * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the
+   * exponential distribution with the input mean.
+   *
+   * @param sc SparkContext used to create the RDD.
+   * @param mean Mean, or 1 / lambda, for the Exponential distribution.
+   * @param numRows Number of Vectors in the RDD.
+   * @param numCols Number of elements in each Vector.
+   * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`)
+   * @param seed Random seed (default: a random long integer).
+   * @return RDD[Vector] with vectors containing i.i.d. samples ~ Exp(mean).
+   */
+  def exponentialVectorRDD(
+      sc: SparkContext,
+      mean: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: Int = 0,
+      seed: Long = Utils.random.nextLong()): RDD[Vector] = {
+    val exponential = new ExponentialGenerator(mean)
+    randomVectorRDD(sc, exponential, numRows, numCols,
+      numPartitionsOrDefault(sc, numPartitions), seed)
+  }
+
+  /**
+   * Java-friendly version of [[RandomRDDs#exponentialVectorRDD]].
+   */
+  def exponentialJavaVectorRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: Int,
+      seed: Long): JavaRDD[Vector] = {
+    exponentialVectorRDD(jsc.sc, mean, numRows, numCols, numPartitions, seed).toJavaRDD()
+  }
+
+  /**
+   * [[RandomRDDs#exponentialJavaVectorRDD]] with the default seed.
+   */
+  def exponentialJavaVectorRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: Int): JavaRDD[Vector] = {
+    exponentialVectorRDD(jsc.sc, mean, numRows, numCols, numPartitions).toJavaRDD()
+  }
+
+  /**
+   * [[RandomRDDs#exponentialJavaVectorRDD]] with the default number of partitions
+   * and the default seed.
+   */
+  def exponentialJavaVectorRDD(
+      jsc: JavaSparkContext,
+      mean: Double,
+      numRows: Long,
+      numCols: Int): JavaRDD[Vector] = {
+    exponentialVectorRDD(jsc.sc, mean, numRows, numCols).toJavaRDD()
+  }
+
+
+  /**
+   * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the
+   * gamma distribution with the input shape and scale.
+   *
+   * @param sc SparkContext used to create the RDD.
+   * @param shape shape parameter (> 0) for the gamma distribution.
+   * @param scale scale parameter (> 0) for the gamma distribution. 
+   * @param numRows Number of Vectors in the RDD.
+   * @param numCols Number of elements in each Vector.
+   * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`)
+   * @param seed Random seed (default: a random long integer).
+   * @return RDD[Vector] with vectors containing i.i.d. samples ~ Exp(mean).
+   */
+  def gammaVectorRDD(
+      sc: SparkContext,
+      shape: Double,
+      scale: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: Int = 0,
+      seed: Long = Utils.random.nextLong()): RDD[Vector] = {
+    val gamma = new GammaGenerator(shape, scale)
+    randomVectorRDD(sc, gamma, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), seed)
+  }
+
+  /**
+   * Java-friendly version of [[RandomRDDs#gammaVectorRDD]].
+   */
+  def gammaJavaVectorRDD(
+      jsc: JavaSparkContext,
+      shape: Double,
+      scale: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: Int,
+      seed: Long): JavaRDD[Vector] = {
+    gammaVectorRDD(jsc.sc, shape, scale, numRows, numCols, numPartitions, seed).toJavaRDD()
+  }
+
+  /**
+   * [[RandomRDDs#gammaJavaVectorRDD]] with the default seed.
+   */
+  def gammaJavaVectorRDD(
+      jsc: JavaSparkContext,
+      shape: Double,
+      scale: Double,
+      numRows: Long,
+      numCols: Int,
+      numPartitions: Int): JavaRDD[Vector] = {
+    gammaVectorRDD(jsc.sc, shape, scale, numRows, numCols, numPartitions).toJavaRDD()
+  }
+
+  /**
+   * [[RandomRDDs#gammaJavaVectorRDD]] with the default number of partitions and the default seed.
+   */
+  def gammaJavaVectorRDD(
+      jsc: JavaSparkContext,
+      shape: Double,
+      scale: Double,
+      numRows: Long,
+      numCols: Int): JavaRDD[Vector] = {
+    gammaVectorRDD(jsc.sc, shape, scale, numRows, numCols).toJavaRDD()
+  }
+
+
   /**
    * :: DeveloperApi ::
    * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
index 90ac252226006..bee951a2e5e26 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
@@ -116,6 +116,7 @@ class ALS private (
 
   /** storage level for user/product in/out links */
   private var intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK
+  private var finalRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK
 
   /**
    * Set the number of blocks for both user blocks and product blocks to parallelize the computation
@@ -204,6 +205,19 @@ class ALS private (
     this
   }
 
+  /**
+   * :: DeveloperApi ::
+   * Sets storage level for final RDDs (user/product used in MatrixFactorizationModel). The default
+   * value is `MEMORY_AND_DISK`. Users can change it to a serialized storage, e.g. 
+   * `MEMORY_AND_DISK_SER` and set `spark.rdd.compress` to `true` to reduce the space requirement,
+   * at the cost of speed.
+   */
+  @DeveloperApi
+  def setFinalRDDStorageLevel(storageLevel: StorageLevel): this.type = {
+    this.finalRDDStorageLevel = storageLevel
+    this
+  }
+
   /**
    * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples.
    * Returns a MatrixFactorizationModel with feature vectors for each user and product.
@@ -307,8 +321,8 @@ class ALS private (
     val usersOut = unblockFactors(users, userOutLinks)
     val productsOut = unblockFactors(products, productOutLinks)
 
-    usersOut.setName("usersOut").persist(StorageLevel.MEMORY_AND_DISK)
-    productsOut.setName("productsOut").persist(StorageLevel.MEMORY_AND_DISK)
+    usersOut.setName("usersOut").persist(finalRDDStorageLevel)
+    productsOut.setName("productsOut").persist(finalRDDStorageLevel)
 
     // Materialize usersOut and productsOut.
     usersOut.count()
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
index f9791c6571782..8ecd5c6ad93c0 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
@@ -45,7 +45,7 @@ class LassoModel (
 /**
  * Train a regression model with L1-regularization using Stochastic Gradient Descent.
  * This solves the l1-regularized least squares regression formulation
- *          f(weights) = 1/n ||A weights-y||^2  + regParam ||weights||_1
+ *          f(weights) = 1/2n ||A weights-y||^2  + regParam ||weights||_1
  * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with
  * its corresponding right hand side label y.
  * See also the documentation for the precise formulation.
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
index c8cad773f5efb..076ba35051c9d 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
@@ -45,7 +45,7 @@ class RidgeRegressionModel (
 /**
  * Train a regression model with L2-regularization using Stochastic Gradient Descent.
  * This solves the l1-regularized least squares regression formulation
- *          f(weights) = 1/n ||A weights-y||^2  + regParam/2 ||weights||^2
+ *          f(weights) = 1/2n ||A weights-y||^2  + regParam/2 ||weights||^2
  * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with
  * its corresponding right hand side label y.
  * See also the documentation for the precise formulation.
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala
index 8db0442a7a569..b549b7c475fc3 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala
@@ -22,7 +22,6 @@ import scala.reflect.ClassTag
 import org.apache.spark.Logging
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.mllib.linalg.Vector
-import org.apache.spark.streaming.StreamingContext._
 import org.apache.spark.streaming.dstream.DStream
 
 /**
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala
new file mode 100644
index 0000000000000..bc7f6c5197ac7
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala
@@ -0,0 +1,100 @@
+/*
+ * 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.mllib.stat.impl
+
+import breeze.linalg.{DenseVector => DBV, DenseMatrix => DBM, diag, max, eigSym}
+
+import org.apache.spark.mllib.util.MLUtils
+
+/**
+ * This class provides basic functionality for a Multivariate Gaussian (Normal) Distribution. In
+ * the event that the covariance matrix is singular, the density will be computed in a
+ * reduced dimensional subspace under which the distribution is supported.
+ * (see [[http://en.wikipedia.org/wiki/Multivariate_normal_distribution#Degenerate_case]])
+ * 
+ * @param mu The mean vector of the distribution
+ * @param sigma The covariance matrix of the distribution
+ */
+private[mllib] class MultivariateGaussian(
+    val mu: DBV[Double], 
+    val sigma: DBM[Double]) extends Serializable {
+
+  /**
+   * Compute distribution dependent constants:
+   *    rootSigmaInv = D^(-1/2) * U, where sigma = U * D * U.t
+   *    u = (2*pi)^(-k/2) * det(sigma)^(-1/2) 
+   */
+  private val (rootSigmaInv: DBM[Double], u: Double) = calculateCovarianceConstants
+  
+  /** Returns density of this multivariate Gaussian at given point, x */
+  def pdf(x: DBV[Double]): Double = {
+    val delta = x - mu
+    val v = rootSigmaInv * delta
+    u * math.exp(v.t * v * -0.5)
+  }
+  
+  /**
+   * Calculate distribution dependent components used for the density function:
+   *    pdf(x) = (2*pi)^(-k/2) * det(sigma)^(-1/2) * exp( (-1/2) * (x-mu).t * inv(sigma) * (x-mu) )
+   * where k is length of the mean vector.
+   * 
+   * We here compute distribution-fixed parts 
+   *  (2*pi)^(-k/2) * det(sigma)^(-1/2)
+   * and
+   *  D^(-1/2) * U, where sigma = U * D * U.t
+   *  
+   * Both the determinant and the inverse can be computed from the singular value decomposition
+   * of sigma.  Noting that covariance matrices are always symmetric and positive semi-definite,
+   * we can use the eigendecomposition. We also do not compute the inverse directly; noting
+   * that 
+   * 
+   *    sigma = U * D * U.t
+   *    inv(Sigma) = U * inv(D) * U.t 
+   *               = (D^{-1/2} * U).t * (D^{-1/2} * U)
+   * 
+   * and thus
+   * 
+   *    -0.5 * (x-mu).t * inv(Sigma) * (x-mu) = -0.5 * norm(D^{-1/2} * U  * (x-mu))^2
+   *  
+   * To guard against singular covariance matrices, this method computes both the 
+   * pseudo-determinant and the pseudo-inverse (Moore-Penrose).  Singular values are considered
+   * to be non-zero only if they exceed a tolerance based on machine precision, matrix size, and
+   * relation to the maximum singular value (same tolerance used by, e.g., Octave).
+   */
+  private def calculateCovarianceConstants: (DBM[Double], Double) = {
+    val eigSym.EigSym(d, u) = eigSym(sigma) // sigma = u * diag(d) * u.t
+    
+    // For numerical stability, values are considered to be non-zero only if they exceed tol.
+    // This prevents any inverted value from exceeding (eps * n * max(d))^-1
+    val tol = MLUtils.EPSILON * max(d) * d.length
+    
+    try {
+      // pseudo-determinant is product of all non-zero singular values
+      val pdetSigma = d.activeValuesIterator.filter(_ > tol).reduce(_ * _)
+      
+      // calculate the root-pseudo-inverse of the diagonal matrix of singular values 
+      // by inverting the square root of all non-zero values
+      val pinvS = diag(new DBV(d.map(v => if (v > tol) math.sqrt(1.0 / v) else 0.0).toArray))
+    
+      (pinvS * u, math.pow(2.0 * math.Pi, -mu.length / 2.0) * math.pow(pdetSigma, -0.5))
+    } catch {
+      case uex: UnsupportedOperationException =>
+        throw new IllegalArgumentException("Covariance matrix has no non-zero singular values")
+    }
+  }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
index 73e7e32c6db31..b3e8ed9af8c51 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
@@ -64,13 +64,6 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo
     val rfModel = rf.run(input)
     rfModel.trees(0)
   }
-
-  /**
-   * Trains a decision tree model over an RDD. This is deprecated because it hides the static
-   * methods with the same name in Java.
-   */
-  @deprecated("Please use DecisionTree.run instead.", "1.2.0")
-  def train(input: RDD[LabeledPoint]): DecisionTreeModel = run(input)
 }
 
 object DecisionTree extends Serializable with Logging {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala
index 7ce9fa6f86c42..55213e695638c 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala
@@ -20,6 +20,7 @@ package org.apache.spark.mllib.tree.loss
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.mllib.regression.LabeledPoint
 import org.apache.spark.mllib.tree.model.TreeEnsembleModel
+import org.apache.spark.mllib.util.MLUtils
 import org.apache.spark.rdd.RDD
 
 /**
@@ -61,13 +62,8 @@ object LogLoss extends Loss {
     data.map { case point =>
       val prediction = model.predict(point.features)
       val margin = 2.0 * point.label * prediction
-      // The following are equivalent to 2.0 * log(1 + exp(-margin)) but are more numerically
-      // stable.
-      if (margin >= 0) {
-        2.0 * math.log1p(math.exp(-margin))
-      } else {
-        2.0 * (-margin + math.log1p(math.exp(margin)))
-      }
+      // The following is equivalent to 2.0 * log(1 + exp(-margin)) but more numerically stable.
+      2.0 * MLUtils.log1pExp(-margin)
     }.mean()
   }
 }
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
index b0d05ae33e1b5..5d6ddd47f67d6 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
@@ -19,8 +19,7 @@ package org.apache.spark.mllib.util
 
 import scala.reflect.ClassTag
 
-import breeze.linalg.{DenseVector => BDV, SparseVector => BSV,
-  squaredDistance => breezeSquaredDistance}
+import breeze.linalg.{DenseVector => BDV, SparseVector => BSV}
 
 import org.apache.spark.annotation.Experimental
 import org.apache.spark.SparkContext
@@ -28,7 +27,7 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.rdd.PartitionwiseSampledRDD
 import org.apache.spark.util.random.BernoulliCellSampler
 import org.apache.spark.mllib.regression.LabeledPoint
-import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors}
+import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, Vectors}
 import org.apache.spark.mllib.linalg.BLAS.dot
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming.StreamingContext
@@ -39,7 +38,7 @@ import org.apache.spark.streaming.dstream.DStream
  */
 object MLUtils {
 
-  private[util] lazy val EPSILON = {
+  private[mllib] lazy val EPSILON = {
     var eps = 1.0
     while ((1.0 + (eps / 2.0)) != 1.0) {
       eps /= 2.0
@@ -154,10 +153,12 @@ object MLUtils {
   def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) {
     // TODO: allow to specify label precision and feature precision.
     val dataStr = data.map { case LabeledPoint(label, features) =>
-      val featureStrings = features.toBreeze.activeIterator.map { case (i, v) =>
-        s"${i + 1}:$v"
+      val sb = new StringBuilder(label.toString)
+      features.foreachActive { case (i, v) =>
+        sb += ' '
+        sb ++= s"${i + 1}:$v"
       }
-      (Iterator(label) ++ featureStrings).mkString(" ")
+      sb.mkString
     }
     dataStr.saveAsTextFile(dir)
   }
@@ -264,7 +265,7 @@ object MLUtils {
     }
     Vectors.fromBreeze(vector1)
   }
-
+ 
   /**
    * Returns the squared Euclidean distance between two vectors. The following formula will be used
    * if it does not introduce too much numerical error:
@@ -314,13 +315,27 @@ object MLUtils {
       val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dotValue)) /
         (sqDist + EPSILON)
       if (precisionBound2 > precision) {
-        // TODO: breezeSquaredDistance is slow,
-        // so we should replace it with our own implementation.
-        sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze)
+        sqDist = Vectors.sqdist(v1, v2)
       }
     } else {
-      sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze)
+      sqDist = Vectors.sqdist(v1, v2)
     }
     sqDist
   }
+
+  /**
+   * When `x` is positive and large, computing `math.log(1 + math.exp(x))` will lead to arithmetic
+   * overflow. This will happen when `x > 709.78` which is not a very large number.
+   * It can be addressed by rewriting the formula into `x + math.log1p(math.exp(-x))` when `x > 0`.
+   *
+   * @param x a floating-point value as input.
+   * @return the result of `math.log(1 + math.exp(x))`.
+   */
+  private[mllib] def log1pExp(x: Double): Double = {
+    if (x > 0) {
+      x + math.log1p(math.exp(-x))
+    } else {
+      math.log1p(math.exp(x))
+    }
+  }
 }
diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java
index 064263e02cd11..fbc26167ce66f 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java
@@ -49,6 +49,7 @@ public void tearDown() {
   public void tfIdf() {
     // The tests are to check Java compatibility.
     HashingTF tf = new HashingTF();
+    @SuppressWarnings("unchecked")
     JavaRDD> documents = sc.parallelize(Lists.newArrayList(
       Lists.newArrayList("this is a sentence".split(" ")),
       Lists.newArrayList("this is another sentence".split(" ")),
@@ -68,6 +69,7 @@ public void tfIdf() {
   public void tfIdfMinimumDocumentFrequency() {
     // The tests are to check Java compatibility.
     HashingTF tf = new HashingTF();
+    @SuppressWarnings("unchecked")
     JavaRDD> documents = sc.parallelize(Lists.newArrayList(
       Lists.newArrayList("this is a sentence".split(" ")),
       Lists.newArrayList("this is another sentence".split(" ")),
diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java
new file mode 100644
index 0000000000000..704d484d0b585
--- /dev/null
+++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java
@@ -0,0 +1,163 @@
+/*
+ * 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.mllib.linalg;
+
+import static org.junit.Assert.*;
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.util.Random;
+
+public class JavaMatricesSuite implements Serializable {
+
+    @Test
+    public void randMatrixConstruction() {
+        Random rng = new Random(24);
+        Matrix r = Matrices.rand(3, 4, rng);
+        rng.setSeed(24);
+        DenseMatrix dr = DenseMatrix.rand(3, 4, rng);
+        assertArrayEquals(r.toArray(), dr.toArray(), 0.0);
+
+        rng.setSeed(24);
+        Matrix rn = Matrices.randn(3, 4, rng);
+        rng.setSeed(24);
+        DenseMatrix drn = DenseMatrix.randn(3, 4, rng);
+        assertArrayEquals(rn.toArray(), drn.toArray(), 0.0);
+
+        rng.setSeed(24);
+        Matrix s = Matrices.sprand(3, 4, 0.5, rng);
+        rng.setSeed(24);
+        SparseMatrix sr = SparseMatrix.sprand(3, 4, 0.5, rng);
+        assertArrayEquals(s.toArray(), sr.toArray(), 0.0);
+
+        rng.setSeed(24);
+        Matrix sn = Matrices.sprandn(3, 4, 0.5, rng);
+        rng.setSeed(24);
+        SparseMatrix srn = SparseMatrix.sprandn(3, 4, 0.5, rng);
+        assertArrayEquals(sn.toArray(), srn.toArray(), 0.0);
+    }
+
+    @Test
+    public void identityMatrixConstruction() {
+        Matrix r = Matrices.eye(2);
+        DenseMatrix dr = DenseMatrix.eye(2);
+        SparseMatrix sr = SparseMatrix.speye(2);
+        assertArrayEquals(r.toArray(), dr.toArray(), 0.0);
+        assertArrayEquals(sr.toArray(), dr.toArray(), 0.0);
+        assertArrayEquals(r.toArray(), new double[]{1.0, 0.0, 0.0, 1.0}, 0.0);
+    }
+
+    @Test
+    public void diagonalMatrixConstruction() {
+        Vector v = Vectors.dense(1.0, 0.0, 2.0);
+        Vector sv = Vectors.sparse(3, new int[]{0, 2}, new double[]{1.0, 2.0});
+
+        Matrix m = Matrices.diag(v);
+        Matrix sm = Matrices.diag(sv);
+        DenseMatrix d = DenseMatrix.diag(v);
+        DenseMatrix sd = DenseMatrix.diag(sv);
+        SparseMatrix s = SparseMatrix.diag(v);
+        SparseMatrix ss = SparseMatrix.diag(sv);
+
+        assertArrayEquals(m.toArray(), sm.toArray(), 0.0);
+        assertArrayEquals(d.toArray(), sm.toArray(), 0.0);
+        assertArrayEquals(d.toArray(), sd.toArray(), 0.0);
+        assertArrayEquals(sd.toArray(), s.toArray(), 0.0);
+        assertArrayEquals(s.toArray(), ss.toArray(), 0.0);
+        assertArrayEquals(s.values(), ss.values(), 0.0);
+        assert(s.values().length == 2);
+        assert(ss.values().length == 2);
+        assert(s.colPtrs().length == 4);
+        assert(ss.colPtrs().length == 4);
+    }
+
+    @Test
+    public void zerosMatrixConstruction() {
+        Matrix z = Matrices.zeros(2, 2);
+        Matrix one = Matrices.ones(2, 2);
+        DenseMatrix dz = DenseMatrix.zeros(2, 2);
+        DenseMatrix done = DenseMatrix.ones(2, 2);
+
+        assertArrayEquals(z.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0);
+        assertArrayEquals(dz.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0);
+        assertArrayEquals(one.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0);
+        assertArrayEquals(done.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0);
+    }
+
+    @Test
+    public void sparseDenseConversion() {
+        int m = 3;
+        int n = 2;
+        double[] values = new double[]{1.0, 2.0, 4.0, 5.0};
+        double[] allValues = new double[]{1.0, 2.0, 0.0, 0.0, 4.0, 5.0};
+        int[] colPtrs = new int[]{0, 2, 4};
+        int[] rowIndices = new int[]{0, 1, 1, 2};
+
+        SparseMatrix spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values);
+        DenseMatrix deMat1 = new DenseMatrix(m, n, allValues);
+
+        SparseMatrix spMat2 = deMat1.toSparse();
+        DenseMatrix deMat2 = spMat1.toDense();
+
+        assertArrayEquals(spMat1.toArray(), spMat2.toArray(), 0.0);
+        assertArrayEquals(deMat1.toArray(), deMat2.toArray(), 0.0);
+    }
+
+    @Test
+    public void concatenateMatrices() {
+        int m = 3;
+        int n = 2;
+
+        Random rng = new Random(42);
+        SparseMatrix spMat1 = SparseMatrix.sprand(m, n, 0.5, rng);
+        rng.setSeed(42);
+        DenseMatrix deMat1 = DenseMatrix.rand(m, n, rng);
+        Matrix deMat2 = Matrices.eye(3);
+        Matrix spMat2 = Matrices.speye(3);
+        Matrix deMat3 = Matrices.eye(2);
+        Matrix spMat3 = Matrices.speye(2);
+
+        Matrix spHorz = Matrices.horzcat(new Matrix[]{spMat1, spMat2});
+        Matrix deHorz1 = Matrices.horzcat(new Matrix[]{deMat1, deMat2});
+        Matrix deHorz2 = Matrices.horzcat(new Matrix[]{spMat1, deMat2});
+        Matrix deHorz3 = Matrices.horzcat(new Matrix[]{deMat1, spMat2});
+
+        assert(deHorz1.numRows() == 3);
+        assert(deHorz2.numRows() == 3);
+        assert(deHorz3.numRows() == 3);
+        assert(spHorz.numRows() == 3);
+        assert(deHorz1.numCols() == 5);
+        assert(deHorz2.numCols() == 5);
+        assert(deHorz3.numCols() == 5);
+        assert(spHorz.numCols() == 5);
+
+        Matrix spVert = Matrices.vertcat(new Matrix[]{spMat1, spMat3});
+        Matrix deVert1 = Matrices.vertcat(new Matrix[]{deMat1, deMat3});
+        Matrix deVert2 = Matrices.vertcat(new Matrix[]{spMat1, deMat3});
+        Matrix deVert3 = Matrices.vertcat(new Matrix[]{deMat1, spMat3});
+
+        assert(deVert1.numRows() == 5);
+        assert(deVert2.numRows() == 5);
+        assert(deVert3.numRows() == 5);
+        assert(spVert.numRows() == 5);
+        assert(deVert1.numCols() == 2);
+        assert(deVert2.numCols() == 2);
+        assert(deVert3.numCols() == 2);
+        assert(spVert.numCols() == 2);
+    }
+}
diff --git a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java
index a725736ca1a58..fcc13c00cbdc5 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java
@@ -69,6 +69,21 @@ public void testNormalRDD() {
     }
   }
 
+  @Test
+  public void testLNormalRDD() {
+    double mean = 4.0;
+    double std = 2.0;
+    long m = 1000L;
+    int p = 2;
+    long seed = 1L;
+    JavaDoubleRDD rdd1 = logNormalJavaRDD(sc, mean, std, m);
+    JavaDoubleRDD rdd2 = logNormalJavaRDD(sc, mean, std, m, p);
+    JavaDoubleRDD rdd3 = logNormalJavaRDD(sc, mean, std, m, p, seed);
+    for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) {
+      Assert.assertEquals(m, rdd.count());
+    }
+  }
+
   @Test
   public void testPoissonRDD() {
     double mean = 2.0;
@@ -83,6 +98,36 @@ public void testPoissonRDD() {
     }
   }
 
+  @Test
+  public void testExponentialRDD() {
+    double mean = 2.0;
+    long m = 1000L;
+    int p = 2;
+    long seed = 1L;
+    JavaDoubleRDD rdd1 = exponentialJavaRDD(sc, mean, m);
+    JavaDoubleRDD rdd2 = exponentialJavaRDD(sc, mean, m, p);
+    JavaDoubleRDD rdd3 = exponentialJavaRDD(sc, mean, m, p, seed);
+    for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) {
+      Assert.assertEquals(m, rdd.count());
+    }
+  }
+
+  @Test
+  public void testGammaRDD() {
+    double shape = 1.0;
+    double scale = 2.0;
+    long m = 1000L;
+    int p = 2;
+    long seed = 1L;
+    JavaDoubleRDD rdd1 = gammaJavaRDD(sc, shape, scale, m);
+    JavaDoubleRDD rdd2 = gammaJavaRDD(sc, shape, scale, m, p);
+    JavaDoubleRDD rdd3 = gammaJavaRDD(sc, shape, scale, m, p, seed);
+    for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) {
+      Assert.assertEquals(m, rdd.count());
+    }
+  }
+
+
   @Test
   @SuppressWarnings("unchecked")
   public void testUniformVectorRDD() {
@@ -115,6 +160,24 @@ public void testNormalVectorRDD() {
     }
   }
 
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testLogNormalVectorRDD() {
+    double mean = 4.0;
+    double std = 2.0;  
+    long m = 100L;
+    int n = 10;
+    int p = 2;
+    long seed = 1L;
+    JavaRDD rdd1 = logNormalJavaVectorRDD(sc, mean, std, m, n);
+    JavaRDD rdd2 = logNormalJavaVectorRDD(sc, mean, std, m, n, p);
+    JavaRDD rdd3 = logNormalJavaVectorRDD(sc, mean, std, m, n, p, seed);
+    for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) {
+      Assert.assertEquals(m, rdd.count());
+      Assert.assertEquals(n, rdd.first().size());
+    }
+  }
+
   @Test
   @SuppressWarnings("unchecked")
   public void testPoissonVectorRDD() {
@@ -131,4 +194,40 @@ public void testPoissonVectorRDD() {
       Assert.assertEquals(n, rdd.first().size());
     }
   }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testExponentialVectorRDD() {
+    double mean = 2.0;
+    long m = 100L;
+    int n = 10;
+    int p = 2;
+    long seed = 1L;
+    JavaRDD rdd1 = exponentialJavaVectorRDD(sc, mean, m, n);
+    JavaRDD rdd2 = exponentialJavaVectorRDD(sc, mean, m, n, p);
+    JavaRDD rdd3 = exponentialJavaVectorRDD(sc, mean, m, n, p, seed);
+    for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) {
+      Assert.assertEquals(m, rdd.count());
+      Assert.assertEquals(n, rdd.first().size());
+    }
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testGammaVectorRDD() {
+    double shape = 1.0;
+    double scale = 2.0;
+    long m = 100L;
+    int n = 10;
+    int p = 2;
+    long seed = 1L;
+    JavaRDD rdd1 = gammaJavaVectorRDD(sc, shape, scale, m, n);
+    JavaRDD rdd2 = gammaJavaVectorRDD(sc, shape, scale, m, n, p);
+    JavaRDD rdd3 = gammaJavaVectorRDD(sc, shape, scale, m, n, p, seed);
+    for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) {
+      Assert.assertEquals(m, rdd.count());
+      Assert.assertEquals(n, rdd.first().size());
+    }
+  }
+
 }
diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties
index a469badf603c6..9697237bfa1a3 100644
--- a/mllib/src/test/resources/log4j.properties
+++ b/mllib/src/test/resources/log4j.properties
@@ -15,10 +15,10 @@
 # limitations under the License.
 #
 
-# Set everything to be logged to the file core/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/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
index 4e812994405b3..94b0e00f37267 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
@@ -178,15 +178,16 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with M
     // Use half as many iterations as the previous test.
     val lr = new LogisticRegressionWithSGD().setIntercept(true)
     lr.optimizer.
-      setStepSize(10.0).
+      setStepSize(1.0).
       setNumIterations(10).
       setRegParam(1.0)
 
     val model = lr.run(testRDD, initialWeights)
 
     // Test the weights
-    assert(model.weights(0) ~== -430000.0 relTol 20000.0)
-    assert(model.intercept ~== 370000.0 relTol 20000.0)
+    // With regularization, the resulting weights will be smaller.
+    assert(model.weights(0) ~== -0.14 relTol 0.02)
+    assert(model.intercept ~== 0.25 relTol 0.02)
 
     val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17)
     val validationRDD = sc.parallelize(validationData, 2)
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala
new file mode 100644
index 0000000000000..23feb82874b70
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala
@@ -0,0 +1,78 @@
+/*
+ * 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.mllib.clustering
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.mllib.linalg.{Vectors, Matrices}
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.mllib.util.TestingUtils._
+
+class GMMExpectationMaximizationSuite extends FunSuite with MLlibTestSparkContext {
+  test("single cluster") {
+    val data = sc.parallelize(Array(
+      Vectors.dense(6.0, 9.0),
+      Vectors.dense(5.0, 10.0),
+      Vectors.dense(4.0, 11.0)
+    ))
+    
+    // expectations
+    val Ew = 1.0
+    val Emu = Vectors.dense(5.0, 10.0)
+    val Esigma = Matrices.dense(2, 2, Array(2.0 / 3.0, -2.0 / 3.0, -2.0 / 3.0, 2.0 / 3.0))
+    
+    val gmm = new GaussianMixtureEM().setK(1).run(data)
+                
+    assert(gmm.weight(0) ~== Ew absTol 1E-5)
+    assert(gmm.mu(0) ~== Emu absTol 1E-5)
+    assert(gmm.sigma(0) ~== Esigma absTol 1E-5)
+  }
+  
+  test("two clusters") {
+    val data = sc.parallelize(Array(
+      Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220),
+      Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118),
+      Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322),
+      Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026),
+      Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734)
+    ))
+  
+    // we set an initial gaussian to induce expected results
+    val initialGmm = new GaussianMixtureModel(
+      Array(0.5, 0.5),
+      Array(Vectors.dense(-1.0), Vectors.dense(1.0)),
+      Array(Matrices.dense(1, 1, Array(1.0)), Matrices.dense(1, 1, Array(1.0)))
+    )
+    
+    val Ew = Array(1.0 / 3.0, 2.0 / 3.0)
+    val Emu = Array(Vectors.dense(-4.3673), Vectors.dense(5.1604))
+    val Esigma = Array(Matrices.dense(1, 1, Array(1.1098)), Matrices.dense(1, 1, Array(0.86644)))
+    
+    val gmm = new GaussianMixtureEM()
+      .setK(2)
+      .setInitialModel(initialGmm)
+      .run(data)
+      
+    assert(gmm.weight(0) ~== Ew(0) absTol 1E-3)
+    assert(gmm.weight(1) ~== Ew(1) absTol 1E-3)
+    assert(gmm.mu(0) ~== Emu(0) absTol 1E-3)
+    assert(gmm.mu(1) ~== Emu(1) absTol 1E-3)
+    assert(gmm.sigma(0) ~== Esigma(0) absTol 1E-3)
+    assert(gmm.sigma(1) ~== Esigma(1) absTol 1E-3)
+  }
+}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala
index 8a18e2971cab6..e0224f960cc43 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala
@@ -124,4 +124,40 @@ class BinaryClassificationMetricsSuite extends FunSuite with MLlibTestSparkConte
 
     validateMetrics(metrics, thresholds, rocCurve, prCurve, f1, f2, precisions, recalls)
   }
+
+  test("binary evaluation metrics with downsampling") {
+    val scoreAndLabels = Seq(
+      (0.1, 0.0), (0.2, 0.0), (0.3, 1.0), (0.4, 0.0), (0.5, 0.0),
+      (0.6, 1.0), (0.7, 1.0), (0.8, 0.0), (0.9, 1.0))
+
+    val scoreAndLabelsRDD = sc.parallelize(scoreAndLabels, 1)
+
+    val original = new BinaryClassificationMetrics(scoreAndLabelsRDD)
+    val originalROC = original.roc().collect().sorted.toList
+    // Add 2 for (0,0) and (1,1) appended at either end
+    assert(2 + scoreAndLabels.size == originalROC.size)
+    assert(
+      List(
+        (0.0, 0.0), (0.0, 0.25), (0.2, 0.25), (0.2, 0.5), (0.2, 0.75),
+        (0.4, 0.75), (0.6, 0.75), (0.6, 1.0), (0.8, 1.0), (1.0, 1.0),
+        (1.0, 1.0)
+      ) ==
+      originalROC)
+
+    val numBins = 4
+
+    val downsampled = new BinaryClassificationMetrics(scoreAndLabelsRDD, numBins)
+    val downsampledROC = downsampled.roc().collect().sorted.toList
+    assert(
+      // May have to add 1 if the sample factor didn't divide evenly
+      2 + (numBins + (if (scoreAndLabels.size % numBins == 0) 0 else 1)) ==
+      downsampledROC.size)
+    assert(
+      List(
+        (0.0, 0.0), (0.2, 0.25), (0.2, 0.75), (0.6, 0.75), (0.8, 1.0),
+        (1.0, 1.0), (1.0, 1.0)
+      ) ==
+      downsampledROC)
+  }
+
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala
index 30147e7fd948f..0a5cad7caf8e4 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala
@@ -19,8 +19,7 @@ package org.apache.spark.mllib.feature
 
 import org.scalatest.FunSuite
 
-import org.apache.spark.SparkContext._
-import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors}
+import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors, Vector}
 import org.apache.spark.mllib.util.MLlibTestSparkContext
 import org.apache.spark.mllib.util.TestingUtils._
 
@@ -41,18 +40,26 @@ class IDFSuite extends FunSuite with MLlibTestSparkContext {
       math.log((m + 1.0) / (x + 1.0))
     })
     assert(model.idf ~== expected absTol 1e-12)
-    val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap()
-    assert(tfidf.size === 3)
-    val tfidf0 = tfidf(0L).asInstanceOf[SparseVector]
-    assert(tfidf0.indices === Array(1, 3))
-    assert(Vectors.dense(tfidf0.values) ~==
-      Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12)
-    val tfidf1 = tfidf(1L).asInstanceOf[DenseVector]
-    assert(Vectors.dense(tfidf1.values) ~==
-      Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12)
-    val tfidf2 = tfidf(2L).asInstanceOf[SparseVector]
-    assert(tfidf2.indices === Array(1))
-    assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12)
+
+    val assertHelper = (tfidf: Array[Vector]) => {
+      assert(tfidf.size === 3)
+      val tfidf0 = tfidf(0).asInstanceOf[SparseVector]
+      assert(tfidf0.indices === Array(1, 3))
+      assert(Vectors.dense(tfidf0.values) ~==
+          Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12)
+      val tfidf1 = tfidf(1).asInstanceOf[DenseVector]
+      assert(Vectors.dense(tfidf1.values) ~==
+          Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12)
+      val tfidf2 = tfidf(2).asInstanceOf[SparseVector]
+      assert(tfidf2.indices === Array(1))
+      assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12)
+    }
+    // Transforms a RDD
+    val tfidf = model.transform(termFrequencies).collect()
+    assertHelper(tfidf)
+    // Transforms local vectors
+    val localTfidf = localTermFrequencies.map(model.transform(_)).toArray
+    assertHelper(localTfidf)
   }
 
   test("idf minimum document frequency filtering") {
@@ -74,18 +81,26 @@ class IDFSuite extends FunSuite with MLlibTestSparkContext {
       }
     })
     assert(model.idf ~== expected absTol 1e-12)
-    val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap()
-    assert(tfidf.size === 3)
-    val tfidf0 = tfidf(0L).asInstanceOf[SparseVector]
-    assert(tfidf0.indices === Array(1, 3))
-    assert(Vectors.dense(tfidf0.values) ~==
-      Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12)
-    val tfidf1 = tfidf(1L).asInstanceOf[DenseVector]
-    assert(Vectors.dense(tfidf1.values) ~==
-      Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12)
-    val tfidf2 = tfidf(2L).asInstanceOf[SparseVector]
-    assert(tfidf2.indices === Array(1))
-    assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12)
+
+    val assertHelper = (tfidf: Array[Vector]) => {
+      assert(tfidf.size === 3)
+      val tfidf0 = tfidf(0).asInstanceOf[SparseVector]
+      assert(tfidf0.indices === Array(1, 3))
+      assert(Vectors.dense(tfidf0.values) ~==
+          Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12)
+      val tfidf1 = tfidf(1).asInstanceOf[DenseVector]
+      assert(Vectors.dense(tfidf1.values) ~==
+          Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12)
+      val tfidf2 = tfidf(2).asInstanceOf[SparseVector]
+      assert(tfidf2.indices === Array(1))
+      assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12)
+    }
+    // Transforms a RDD
+    val tfidf = model.transform(termFrequencies).collect()
+    assertHelper(tfidf)
+    // Transforms local vectors
+    val localTfidf = localTermFrequencies.map(model.transform(_)).toArray
+    assertHelper(localTfidf)
   }
 
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala
index 5d70c914f14b0..771878e925ea7 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala
@@ -127,6 +127,47 @@ class BLASSuite extends FunSuite {
     }
   }
 
+  test("syr") {
+    val dA = new DenseMatrix(4, 4,
+      Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0, 3.1, 4.6, 3.0, 0.8))
+    val x = new DenseVector(Array(0.0, 2.7, 3.5, 2.1))
+    val alpha = 0.15
+
+    val expected = new DenseMatrix(4, 4,
+      Array(0.0, 1.2, 2.2, 3.1, 1.2, 4.2935, 6.7175, 5.4505, 2.2, 6.7175, 3.6375, 4.1025, 3.1,
+        5.4505, 4.1025, 1.4615))
+
+    syr(alpha, x, dA)
+
+    assert(dA ~== expected absTol 1e-15)
+ 
+    val dB =
+      new DenseMatrix(3, 4, Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0))
+
+    withClue("Matrix A must be a symmetric Matrix") {
+      intercept[Exception] {
+        syr(alpha, x, dB)
+      }
+    }
+ 
+    val dC =
+      new DenseMatrix(3, 3, Array(0.0, 1.2, 2.2, 1.2, 3.2, 5.3, 2.2, 5.3, 1.8))
+
+    withClue("Size of vector must match the rank of matrix") {
+      intercept[Exception] {
+        syr(alpha, x, dC)
+      }
+    }
+ 
+    val y = new DenseVector(Array(0.0, 2.7, 3.5, 2.1, 1.5))
+
+    withClue("Size of vector must match the rank of matrix") {
+      intercept[Exception] {
+        syr(alpha, y, dA)
+      }
+    }
+  }
+
   test("gemm") {
 
     val dA =
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala
index 322a0e9242918..a35d0fe389fdd 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala
@@ -43,9 +43,9 @@ class MatricesSuite extends FunSuite {
 
   test("sparse matrix construction") {
     val m = 3
-    val n = 2
+    val n = 4
     val values = Array(1.0, 2.0, 4.0, 5.0)
-    val colPtrs = Array(0, 2, 4)
+    val colPtrs = Array(0, 2, 2, 4, 4)
     val rowIndices = Array(1, 2, 1, 2)
     val mat = Matrices.sparse(m, n, colPtrs, rowIndices, values).asInstanceOf[SparseMatrix]
     assert(mat.numRows === m)
@@ -53,6 +53,13 @@ class MatricesSuite extends FunSuite {
     assert(mat.values.eq(values), "should not copy data")
     assert(mat.colPtrs.eq(colPtrs), "should not copy data")
     assert(mat.rowIndices.eq(rowIndices), "should not copy data")
+
+    val entries: Array[(Int, Int, Double)] = Array((2, 2, 3.0), (1, 0, 1.0), (2, 0, 2.0),
+        (1, 2, 2.0), (2, 2, 2.0), (1, 2, 2.0), (0, 0, 0.0))
+
+    val mat2 = SparseMatrix.fromCOO(m, n, entries)
+    assert(mat.toBreeze === mat2.toBreeze)
+    assert(mat2.values.length == 4)
   }
 
   test("sparse matrix construction with wrong number of elements") {
@@ -117,6 +124,142 @@ class MatricesSuite extends FunSuite {
     assert(sparseMat.values(2) === 10.0)
   }
 
+  test("toSparse, toDense") {
+    val m = 3
+    val n = 2
+    val values = Array(1.0, 2.0, 4.0, 5.0)
+    val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0)
+    val colPtrs = Array(0, 2, 4)
+    val rowIndices = Array(0, 1, 1, 2)
+
+    val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values)
+    val deMat1 = new DenseMatrix(m, n, allValues)
+
+    val spMat2 = deMat1.toSparse()
+    val deMat2 = spMat1.toDense()
+
+    assert(spMat1.toBreeze === spMat2.toBreeze)
+    assert(deMat1.toBreeze === deMat2.toBreeze)
+  }
+
+  test("map, update") {
+    val m = 3
+    val n = 2
+    val values = Array(1.0, 2.0, 4.0, 5.0)
+    val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0)
+    val colPtrs = Array(0, 2, 4)
+    val rowIndices = Array(0, 1, 1, 2)
+
+    val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values)
+    val deMat1 = new DenseMatrix(m, n, allValues)
+    val deMat2 = deMat1.map(_ * 2)
+    val spMat2 = spMat1.map(_ * 2)
+    deMat1.update(_ * 2)
+    spMat1.update(_ * 2)
+
+    assert(spMat1.toArray === spMat2.toArray)
+    assert(deMat1.toArray === deMat2.toArray)
+  }
+
+  test("horzcat, vertcat, eye, speye") {
+    val m = 3
+    val n = 2
+    val values = Array(1.0, 2.0, 4.0, 5.0)
+    val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0)
+    val colPtrs = Array(0, 2, 4)
+    val rowIndices = Array(0, 1, 1, 2)
+
+    val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values)
+    val deMat1 = new DenseMatrix(m, n, allValues)
+    val deMat2 = Matrices.eye(3)
+    val spMat2 = Matrices.speye(3)
+    val deMat3 = Matrices.eye(2)
+    val spMat3 = Matrices.speye(2)
+
+    val spHorz = Matrices.horzcat(Array(spMat1, spMat2))
+    val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2))
+    val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2))
+    val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2))
+
+    val deHorz2 = Matrices.horzcat(Array[Matrix]())
+
+    assert(deHorz1.numRows === 3)
+    assert(spHorz2.numRows === 3)
+    assert(spHorz3.numRows === 3)
+    assert(spHorz.numRows === 3)
+    assert(deHorz1.numCols === 5)
+    assert(spHorz2.numCols === 5)
+    assert(spHorz3.numCols === 5)
+    assert(spHorz.numCols === 5)
+    assert(deHorz2.numRows === 0)
+    assert(deHorz2.numCols === 0)
+    assert(deHorz2.toArray.length === 0)
+
+    assert(deHorz1.toBreeze.toDenseMatrix === spHorz2.toBreeze.toDenseMatrix)
+    assert(spHorz2.toBreeze === spHorz3.toBreeze)
+    assert(spHorz(0, 0) === 1.0)
+    assert(spHorz(2, 1) === 5.0)
+    assert(spHorz(0, 2) === 1.0)
+    assert(spHorz(1, 2) === 0.0)
+    assert(spHorz(1, 3) === 1.0)
+    assert(spHorz(2, 4) === 1.0)
+    assert(spHorz(1, 4) === 0.0)
+    assert(deHorz1(0, 0) === 1.0)
+    assert(deHorz1(2, 1) === 5.0)
+    assert(deHorz1(0, 2) === 1.0)
+    assert(deHorz1(1, 2) == 0.0)
+    assert(deHorz1(1, 3) === 1.0)
+    assert(deHorz1(2, 4) === 1.0)
+    assert(deHorz1(1, 4) === 0.0)
+
+    intercept[IllegalArgumentException] {
+      Matrices.horzcat(Array(spMat1, spMat3))
+    }
+
+    intercept[IllegalArgumentException] {
+      Matrices.horzcat(Array(deMat1, spMat3))
+    }
+
+    val spVert = Matrices.vertcat(Array(spMat1, spMat3))
+    val deVert1 = Matrices.vertcat(Array(deMat1, deMat3))
+    val spVert2 = Matrices.vertcat(Array(spMat1, deMat3))
+    val spVert3 = Matrices.vertcat(Array(deMat1, spMat3))
+    val deVert2 = Matrices.vertcat(Array[Matrix]())
+
+    assert(deVert1.numRows === 5)
+    assert(spVert2.numRows === 5)
+    assert(spVert3.numRows === 5)
+    assert(spVert.numRows === 5)
+    assert(deVert1.numCols === 2)
+    assert(spVert2.numCols === 2)
+    assert(spVert3.numCols === 2)
+    assert(spVert.numCols === 2)
+    assert(deVert2.numRows === 0)
+    assert(deVert2.numCols === 0)
+    assert(deVert2.toArray.length === 0)
+
+    assert(deVert1.toBreeze.toDenseMatrix === spVert2.toBreeze.toDenseMatrix)
+    assert(spVert2.toBreeze === spVert3.toBreeze)
+    assert(spVert(0, 0) === 1.0)
+    assert(spVert(2, 1) === 5.0)
+    assert(spVert(3, 0) === 1.0)
+    assert(spVert(3, 1) === 0.0)
+    assert(spVert(4, 1) === 1.0)
+    assert(deVert1(0, 0) === 1.0)
+    assert(deVert1(2, 1) === 5.0)
+    assert(deVert1(3, 0) === 1.0)
+    assert(deVert1(3, 1) === 0.0)
+    assert(deVert1(4, 1) === 1.0)
+
+    intercept[IllegalArgumentException] {
+      Matrices.vertcat(Array(spMat1, spMat2))
+    }
+
+    intercept[IllegalArgumentException] {
+      Matrices.vertcat(Array(deMat1, spMat2))
+    }
+  }
+
   test("zeros") {
     val mat = Matrices.zeros(2, 3).asInstanceOf[DenseMatrix]
     assert(mat.numRows === 2)
@@ -162,4 +305,29 @@ class MatricesSuite extends FunSuite {
     assert(mat.numCols === 2)
     assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 2.0))
   }
+
+  test("sprand") {
+    val rng = mock[Random]
+    when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0)
+    when(rng.nextDouble()).thenReturn(1.0, 2.0, 3.0, 4.0, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)
+    val mat = SparseMatrix.sprand(4, 4, 0.25, rng)
+    assert(mat.numRows === 4)
+    assert(mat.numCols === 4)
+    assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1))
+    assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0))
+    val mat2 = SparseMatrix.sprand(2, 3, 1.0, rng)
+    assert(mat2.rowIndices.toSeq === Seq(0, 1, 0, 1, 0, 1))
+    assert(mat2.colPtrs.toSeq === Seq(0, 2, 4, 6))
+  }
+
+  test("sprandn") {
+    val rng = mock[Random]
+    when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0)
+    when(rng.nextGaussian()).thenReturn(1.0, 2.0, 3.0, 4.0)
+    val mat = SparseMatrix.sprandn(4, 4, 0.25, rng)
+    assert(mat.numRows === 4)
+    assert(mat.numCols === 4)
+    assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1))
+    assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0))
+  }
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
index f99f01450992a..85ac8ccebfc59 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
@@ -17,7 +17,9 @@
 
 package org.apache.spark.mllib.linalg
 
-import breeze.linalg.{DenseMatrix => BDM}
+import scala.util.Random
+
+import breeze.linalg.{DenseMatrix => BDM, squaredDistance => breezeSquaredDistance}
 import org.scalatest.FunSuite
 
 import org.apache.spark.SparkException
@@ -175,6 +177,33 @@ class VectorsSuite extends FunSuite {
     assert(v.size === x.rows)
   }
 
+  test("sqdist") {
+    val random = new Random()
+    for (m <- 1 until 1000 by 100) {
+      val nnz = random.nextInt(m)
+
+      val indices1 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray
+      val values1 = Array.fill(nnz)(random.nextDouble)
+      val sparseVector1 = Vectors.sparse(m, indices1, values1)
+
+      val indices2 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray
+      val values2 = Array.fill(nnz)(random.nextDouble)
+      val sparseVector2 = Vectors.sparse(m, indices2, values2)
+
+      val denseVector1 = Vectors.dense(sparseVector1.toArray)
+      val denseVector2 = Vectors.dense(sparseVector2.toArray)
+
+      val squaredDist = breezeSquaredDistance(sparseVector1.toBreeze, sparseVector2.toBreeze)
+
+      // SparseVector vs. SparseVector 
+      assert(Vectors.sqdist(sparseVector1, sparseVector2) ~== squaredDist relTol 1E-8) 
+      // DenseVector  vs. SparseVector
+      assert(Vectors.sqdist(denseVector1, sparseVector2) ~== squaredDist relTol 1E-8)
+      // DenseVector  vs. DenseVector
+      assert(Vectors.sqdist(denseVector1, denseVector2) ~== squaredDist relTol 1E-8)
+    }    
+  }
+
   test("foreachActive") {
     val dv = Vectors.dense(0.0, 1.2, 3.1, 0.0)
     val sv = Vectors.sparse(4, Seq((1, 1.2), (2, 3.1), (3, 0.0)))
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala
index 3df7c128af5ab..b792d819fdabb 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.mllib.random
 
+import scala.math
+
 import org.scalatest.FunSuite
 
 import org.apache.spark.util.StatCounter
@@ -25,7 +27,6 @@ import org.apache.spark.util.StatCounter
 class RandomDataGeneratorSuite extends FunSuite {
 
   def apiChecks(gen: RandomDataGenerator[Double]) {
-
     // resetting seed should generate the same sequence of random numbers
     gen.setSeed(42L)
     val array1 = (0 until 1000).map(_ => gen.nextValue())
@@ -79,6 +80,26 @@ class RandomDataGeneratorSuite extends FunSuite {
     distributionChecks(normal, 0.0, 1.0)
   }
 
+  test("LogNormalGenerator") {
+    List((0.0, 1.0), (0.0, 2.0), (2.0, 1.0), (2.0, 2.0)).map {
+      case (mean: Double, vari: Double) =>
+        val normal = new LogNormalGenerator(mean, math.sqrt(vari))
+        apiChecks(normal)
+
+        // mean of log normal = e^(mean + var / 2)
+        val expectedMean = math.exp(mean + 0.5 * vari)
+
+        // variance of log normal = (e^var - 1) * e^(2 * mean + var)
+        val expectedStd = math.sqrt((math.exp(vari) - 1.0) * math.exp(2.0 * mean + vari))
+
+        // since sampling error increases with variance, let's set
+        // the absolute tolerance as a percentage
+        val epsilon = 0.05 * expectedStd * expectedStd
+
+        distributionChecks(normal, expectedMean, expectedStd, epsilon)
+    }
+  }
+
   test("PoissonGenerator") {
     // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced.
     for (mean <- List(1.0, 5.0, 100.0)) {
@@ -87,4 +108,33 @@ class RandomDataGeneratorSuite extends FunSuite {
       distributionChecks(poisson, mean, math.sqrt(mean), 0.1)
     }
   }
+
+  test("ExponentialGenerator") {
+    // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced.
+    for (mean <- List(2.0, 5.0, 10.0, 50.0, 100.0)) {
+      val exponential = new ExponentialGenerator(mean)
+      apiChecks(exponential)
+      // var of exp = lambda^-2 = (1.0 / mean)^-2 = mean^2
+
+      // since sampling error increases with variance, let's set
+      // the absolute tolerance as a percentage
+      val epsilon = 0.05 * mean * mean
+
+      distributionChecks(exponential, mean, mean, epsilon)
+    }
+  }
+
+  test("GammaGenerator") {
+    // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced.
+    List((1.0, 2.0), (2.0, 2.0), (3.0, 2.0), (5.0, 1.0), (9.0, 0.5)).map {
+      case (shape: Double, scale: Double) =>
+        val gamma = new GammaGenerator(shape, scale)
+        apiChecks(gamma)
+        // mean of gamma = shape * scale
+        val expectedMean = shape * scale
+        // var of gamma = shape * scale^2
+        val expectedStd = math.sqrt(shape * scale * scale)
+        distributionChecks(gamma, expectedMean, expectedStd, 0.1)
+    }
+  }
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala
index ea5889b3ecd5e..6395188a0842a 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala
@@ -110,7 +110,19 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa
   test("randomRDD for different distributions") {
     val size = 100000L
     val numPartitions = 10
+
+    //  mean of log normal = e^(mean + var / 2)
+    val logNormalMean = math.exp(0.5)
+    // variance of log normal = (e^var - 1) * e^(2 * mean + var)
+    val logNormalStd = math.sqrt((math.E - 1.0) * math.E)
+    val gammaScale = 1.0
+    val gammaShape = 2.0
+    // mean of gamma = shape * scale
+    val gammaMean = gammaShape * gammaScale
+    // var of gamma = shape * scale^2
+    val gammaStd = math.sqrt(gammaShape * gammaScale * gammaScale)
     val poissonMean = 100.0
+    val exponentialMean = 1.0
 
     for (seed <- 0 until 5) {
       val uniform = RandomRDDs.uniformRDD(sc, size, numPartitions, seed)
@@ -119,8 +131,18 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa
       val normal = RandomRDDs.normalRDD(sc, size, numPartitions, seed)
       testGeneratedRDD(normal, size, numPartitions, 0.0, 1.0)
 
+      val logNormal = RandomRDDs.logNormalRDD(sc, 0.0, 1.0, size, numPartitions, seed)
+      testGeneratedRDD(logNormal, size, numPartitions, logNormalMean, logNormalStd, 0.1)
+
       val poisson = RandomRDDs.poissonRDD(sc, poissonMean, size, numPartitions, seed)
       testGeneratedRDD(poisson, size, numPartitions, poissonMean, math.sqrt(poissonMean), 0.1)
+
+      val exponential = RandomRDDs.exponentialRDD(sc, exponentialMean, size, numPartitions, seed)
+      testGeneratedRDD(exponential, size, numPartitions, exponentialMean, exponentialMean, 0.1)
+
+      val gamma = RandomRDDs.gammaRDD(sc, gammaShape, gammaScale, size, numPartitions, seed)
+      testGeneratedRDD(gamma, size, numPartitions, gammaMean, gammaStd, 0.1)
+
     }
 
     // mock distribution to check that partitions have unique seeds
@@ -132,7 +154,19 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa
     val rows = 1000L
     val cols = 100
     val parts = 10
+
+    //  mean of log normal = e^(mean + var / 2)
+    val logNormalMean = math.exp(0.5)
+    // variance of log normal = (e^var - 1) * e^(2 * mean + var)
+    val logNormalStd = math.sqrt((math.E - 1.0) * math.E)
+    val gammaScale = 1.0
+    val gammaShape = 2.0
+    // mean of gamma = shape * scale
+    val gammaMean = gammaShape * gammaScale
+    // var of gamma = shape * scale^2
+    val gammaStd = math.sqrt(gammaShape * gammaScale * gammaScale)
     val poissonMean = 100.0
+    val exponentialMean = 1.0
 
     for (seed <- 0 until 5) {
       val uniform = RandomRDDs.uniformVectorRDD(sc, rows, cols, parts, seed)
@@ -141,8 +175,17 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa
       val normal = RandomRDDs.normalVectorRDD(sc, rows, cols, parts, seed)
       testGeneratedVectorRDD(normal, rows, cols, parts, 0.0, 1.0)
 
+      val logNormal = RandomRDDs.logNormalVectorRDD(sc, 0.0, 1.0, rows, cols, parts, seed)
+      testGeneratedVectorRDD(logNormal, rows, cols, parts, logNormalMean, logNormalStd, 0.1)
+
       val poisson = RandomRDDs.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed)
       testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1)
+
+      val exponential = RandomRDDs.exponentialVectorRDD(sc, exponentialMean, rows, cols, parts, seed)
+      testGeneratedVectorRDD(exponential, rows, cols, parts, exponentialMean, exponentialMean, 0.1)
+
+      val gamma = RandomRDDs.gammaVectorRDD(sc, gammaShape, gammaScale, rows, cols, parts, seed)
+      testGeneratedVectorRDD(gamma, rows, cols, parts, gammaMean, gammaStd, 0.1)
     }
   }
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
index 603d0ad127b86..f3b7bfda788fa 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
@@ -27,6 +27,7 @@ import org.jblas.DoubleMatrix
 import org.apache.spark.SparkContext._
 import org.apache.spark.mllib.util.MLlibTestSparkContext
 import org.apache.spark.mllib.recommendation.ALS.BlockStats
+import org.apache.spark.storage.StorageLevel
 
 object ALSSuite {
 
@@ -139,6 +140,32 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext {
     assert(u11 != u2)
   }
 
+  test("Storage Level for RDDs in model") {
+    val ratings = sc.parallelize(ALSSuite.generateRatings(10, 20, 5, 0.5, false, false)._1, 2)
+    var storageLevel = StorageLevel.MEMORY_ONLY
+    var model = new ALS()
+      .setRank(5)
+      .setIterations(1)
+      .setLambda(1.0)
+      .setBlocks(2)
+      .setSeed(1)
+      .setFinalRDDStorageLevel(storageLevel)
+      .run(ratings)
+    assert(model.productFeatures.getStorageLevel == storageLevel);
+    assert(model.userFeatures.getStorageLevel == storageLevel);
+    storageLevel = StorageLevel.DISK_ONLY
+    model = new ALS()
+      .setRank(5)
+      .setIterations(1)
+      .setLambda(1.0)
+      .setBlocks(2)
+      .setSeed(1)
+      .setFinalRDDStorageLevel(storageLevel)
+      .run(ratings)
+    assert(model.productFeatures.getStorageLevel == storageLevel);
+    assert(model.userFeatures.getStorageLevel == storageLevel);
+  }
+
   test("negative ids") {
     val data = ALSSuite.generateRatings(50, 50, 2, 0.7, false, false)
     val ratings = sc.parallelize(data._1.map { case Rating(u, p, r) =>
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala
index 03b71301e9ab1..70b43ddb7daf5 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala
@@ -52,7 +52,7 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase {
     // create model
     val model = new StreamingLinearRegressionWithSGD()
       .setInitialWeights(Vectors.dense(0.0, 0.0))
-      .setStepSize(0.1)
+      .setStepSize(0.2)
       .setNumIterations(25)
 
     // generate sequence of simulated data
@@ -84,7 +84,7 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase {
     // create model
     val model = new StreamingLinearRegressionWithSGD()
       .setInitialWeights(Vectors.dense(0.0))
-      .setStepSize(0.1)
+      .setStepSize(0.2)
       .setNumIterations(25)
 
     // generate sequence of simulated data
@@ -118,7 +118,7 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase {
     // create model initialized with true weights
     val model = new StreamingLinearRegressionWithSGD()
       .setInitialWeights(Vectors.dense(10.0, 10.0))
-      .setStepSize(0.1)
+      .setStepSize(0.2)
       .setNumIterations(25)
 
     // generate sequence of simulated data for testing
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussianSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussianSuite.scala
new file mode 100644
index 0000000000000..d58f2587e55aa
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussianSuite.scala
@@ -0,0 +1,70 @@
+/*
+ * 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.mllib.stat.impl
+
+import org.scalatest.FunSuite
+
+import breeze.linalg.{ DenseVector => BDV, DenseMatrix => BDM }
+
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.mllib.util.TestingUtils._
+
+class MultivariateGaussianSuite extends FunSuite with MLlibTestSparkContext {
+  test("univariate") {
+    val x1 = new BDV(Array(0.0))
+    val x2 = new BDV(Array(1.5))
+                     
+    val mu = new BDV(Array(0.0))
+    val sigma1 = new BDM(1, 1, Array(1.0))
+    val dist1 = new MultivariateGaussian(mu, sigma1)
+    assert(dist1.pdf(x1) ~== 0.39894 absTol 1E-5)
+    assert(dist1.pdf(x2) ~== 0.12952 absTol 1E-5)
+    
+    val sigma2 = new BDM(1, 1, Array(4.0))
+    val dist2 = new MultivariateGaussian(mu, sigma2)
+    assert(dist2.pdf(x1) ~== 0.19947 absTol 1E-5)
+    assert(dist2.pdf(x2) ~== 0.15057 absTol 1E-5)
+  }
+  
+  test("multivariate") {
+    val x1 = new BDV(Array(0.0, 0.0))
+    val x2 = new BDV(Array(1.0, 1.0))
+    
+    val mu = new BDV(Array(0.0, 0.0))
+    val sigma1 = new BDM(2, 2, Array(1.0, 0.0, 0.0, 1.0))
+    val dist1 = new MultivariateGaussian(mu, sigma1)
+    assert(dist1.pdf(x1) ~== 0.15915 absTol 1E-5)
+    assert(dist1.pdf(x2) ~== 0.05855 absTol 1E-5)
+    
+    val sigma2 = new BDM(2, 2, Array(4.0, -1.0, -1.0, 2.0))
+    val dist2 = new MultivariateGaussian(mu, sigma2)
+    assert(dist2.pdf(x1) ~== 0.060155 absTol 1E-5)
+    assert(dist2.pdf(x2) ~== 0.033971 absTol 1E-5)
+  }
+  
+  test("multivariate degenerate") {
+    val x1 = new BDV(Array(0.0, 0.0))
+    val x2 = new BDV(Array(1.0, 1.0))
+    
+    val mu = new BDV(Array(0.0, 0.0))
+    val sigma = new BDM(2, 2, Array(1.0, 1.0, 1.0, 1.0))
+    val dist = new MultivariateGaussian(mu, sigma)
+    assert(dist.pdf(x1) ~== 0.11254 absTol 1E-5)
+    assert(dist.pdf(x2) ~== 0.068259 absTol 1E-5)
+  }
+}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala
index df07987093fbf..668fc1d43c5d6 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala
@@ -20,18 +20,17 @@ package org.apache.spark.mllib.util
 import java.io.File
 
 import scala.io.Source
-import scala.math
 
 import org.scalatest.FunSuite
 
-import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNorm,
-  squaredDistance => breezeSquaredDistance}
+import breeze.linalg.{squaredDistance => breezeSquaredDistance}
 import com.google.common.base.Charsets
 import com.google.common.io.Files
 
 import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors}
 import org.apache.spark.mllib.regression.LabeledPoint
 import org.apache.spark.mllib.util.MLUtils._
+import org.apache.spark.mllib.util.TestingUtils._
 import org.apache.spark.util.Utils
 
 class MLUtilsSuite extends FunSuite with MLlibTestSparkContext {
@@ -52,12 +51,27 @@ class MLUtilsSuite extends FunSuite with MLlibTestSparkContext {
       val values = indices.map(i => a(i))
       val v2 = Vectors.sparse(n, indices, values)
       val norm2 = Vectors.norm(v2, 2.0)
+      val v3 = Vectors.sparse(n, indices, indices.map(i => a(i) + 0.5))
+      val norm3 = Vectors.norm(v3, 2.0)
       val squaredDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze)
       val fastSquaredDist1 = fastSquaredDistance(v1, norm1, v2, norm2, precision)
       assert((fastSquaredDist1 - squaredDist) <= precision * squaredDist, s"failed with m = $m")
       val fastSquaredDist2 =
         fastSquaredDistance(v1, norm1, Vectors.dense(v2.toArray), norm2, precision)
       assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m")
+      val squaredDist2 = breezeSquaredDistance(v2.toBreeze, v3.toBreeze)
+      val fastSquaredDist3 =
+        fastSquaredDistance(v2, norm2, v3, norm3, precision)
+      assert((fastSquaredDist3 - squaredDist2) <= precision * squaredDist2, s"failed with m = $m")
+      if (m > 10) { 
+        val v4 = Vectors.sparse(n, indices.slice(0, m - 10),
+          indices.map(i => a(i) + 0.5).slice(0, m - 10))
+        val norm4 = Vectors.norm(v4, 2.0)
+        val squaredDist = breezeSquaredDistance(v2.toBreeze, v4.toBreeze)
+        val fastSquaredDist =
+          fastSquaredDistance(v2, norm2, v4, norm4, precision)
+        assert((fastSquaredDist - squaredDist) <= precision * squaredDist, s"failed with m = $m")
+      }
     }
   }
 
@@ -189,4 +203,12 @@ class MLUtilsSuite extends FunSuite with MLlibTestSparkContext {
     assert(points.collect().toSet === loaded.collect().toSet)
     Utils.deleteRecursively(tempDir)
   }
+
+  test("log1pExp") {
+    assert(log1pExp(76.3) ~== math.log1p(math.exp(76.3)) relTol 1E-10)
+    assert(log1pExp(87296763.234) ~== 87296763.234 relTol 1E-10)
+
+    assert(log1pExp(-13.8) ~== math.log1p(math.exp(-13.8)) absTol 1E-10)
+    assert(log1pExp(-238423789.865) ~== math.log1p(math.exp(-238423789.865)) absTol 1E-10)
+  }
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala
index 30b906aaa3ba4..e957fa5d25f4c 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala
@@ -178,17 +178,17 @@ object TestingUtils {
   implicit class MatrixWithAlmostEquals(val x: Matrix) {
 
     /**
-     * When the difference of two vectors are within eps, returns true; otherwise, returns false.
+     * When the difference of two matrices are within eps, returns true; otherwise, returns false.
      */
     def ~=(r: CompareMatrixRightSide): Boolean = r.fun(x, r.y, r.eps)
 
     /**
-     * When the difference of two vectors are within eps, returns false; otherwise, returns true.
+     * When the difference of two matrices are within eps, returns false; otherwise, returns true.
      */
     def !~=(r: CompareMatrixRightSide): Boolean = !r.fun(x, r.y, r.eps)
 
     /**
-     * Throws exception when the difference of two vectors are NOT within eps;
+     * Throws exception when the difference of two matrices are NOT within eps;
      * otherwise, returns true.
      */
     def ~==(r: CompareMatrixRightSide): Boolean = {
diff --git a/network/common/pom.xml b/network/common/pom.xml
index baca859fa5011..245a96b8c4038 100644
--- a/network/common/pom.xml
+++ b/network/common/pom.xml
@@ -75,11 +75,6 @@
       mockito-all
       test
     
-    
-      org.scalatest
-      scalatest_${scala.binary.version}
-      test
-    
   
 
   
diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java
index 13b37f96f8ce2..6c9178688693f 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java
@@ -37,10 +37,11 @@ public boolean preferDirectBufs() {
 
   /** Connect timeout in milliseconds. Default 120 secs. */
   public int connectionTimeoutMs() {
-    return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000;
+    int defaultTimeout = conf.getInt("spark.network.timeout", 120);
+    return conf.getInt("spark.shuffle.io.connectionTimeout", defaultTimeout) * 1000;
   }
 
-  /** Number of concurrent connections between two nodes for fetching data. **/
+  /** Number of concurrent connections between two nodes for fetching data. */
   public int numConnectionsPerPeer() {
     return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 1);
   }
diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml
index 12468567c3aed..5bfa1ac9c373e 100644
--- a/network/shuffle/pom.xml
+++ b/network/shuffle/pom.xml
@@ -83,11 +83,6 @@
       mockito-all
       test
     
-    
-      org.scalatest
-      scalatest_${scala.binary.version}
-      test
-    
   
 
   
diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java
index dfe0ba0595090..93e6fdd7161fa 100644
--- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java
+++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java
@@ -37,6 +37,7 @@
 import org.apache.spark.network.buffer.ManagedBuffer;
 import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
 import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
 import org.apache.spark.network.util.TransportConf;
 
 /**
@@ -49,7 +50,7 @@
  * the Executor's memory, unlike the IndexShuffleBlockManager.
  */
 public class ExternalShuffleBlockManager {
-  private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockManager.class);
+  private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockManager.class);
 
   // Map containing all registered executors' metadata.
   private final ConcurrentMap executors;
@@ -60,8 +61,9 @@ public class ExternalShuffleBlockManager {
   private final TransportConf conf;
 
   public ExternalShuffleBlockManager(TransportConf conf) {
-    // TODO: Give this thread a name.
-    this(conf, Executors.newSingleThreadExecutor());
+    this(conf, Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner")));
   }
 
   // Allows tests to have more control over when directories are cleaned up.
diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
index 0191fe529e1be..1ad0d72ae5ec5 100644
--- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
+++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
@@ -54,13 +54,13 @@ public class RetryingBlockFetcherSuite {
   @Before
   public void beforeEach() {
     System.setProperty("spark.shuffle.io.maxRetries", "2");
-    System.setProperty("spark.shuffle.io.retryWaitMs", "0");
+    System.setProperty("spark.shuffle.io.retryWait", "0");
   }
 
   @After
   public void afterEach() {
     System.clearProperty("spark.shuffle.io.maxRetries");
-    System.clearProperty("spark.shuffle.io.retryWaitMs");
+    System.clearProperty("spark.shuffle.io.retryWait");
   }
 
   @Test
diff --git a/pom.xml b/pom.xml
index f42257265ede3..703e5c47bf59b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -123,8 +123,10 @@
     2.4.1
     ${hadoop.version}
     0.94.6
+    hbase
     1.4.0
     3.4.5
+    org.spark-project.hive
     
     0.13.1a
     
@@ -143,12 +145,36 @@
     4.2.6
     3.1.1
     ${project.build.directory}/spark-test-classpath.txt
-    64m
-    512m
     2.10.4
     2.10
     ${scala.version}
     org.scala-lang
+    1.8.8
+    1.1.1.6
+
+    
+    compile
+    compile
+    compile
+    compile
+    compile
+
+    
+    ${session.executionRootDirectory}
+
+    64m
+    512m
+    512m
   
 
   
@@ -230,18 +256,6 @@
         false
       
     
-    
-    
-      spark-staging-1038
-      Spark 1.2.0 Staging (1038)
-      https://repository.apache.org/content/repositories/orgapachespark-1038/
-      
-        true
-      
-      
-        false
-      
-    
   
   
     
@@ -255,21 +269,20 @@
       
     
   
-
   
-  
+    
     
       org.spark-project.spark
       unused
       1.0.0
     
     
     
       org.codehaus.groovy
@@ -277,6 +290,15 @@
       2.3.7
       provided
     
+    
+    
+      org.scalatest
+      scalatest_${scala.binary.version}
+      test
+    
   
   
     
@@ -371,11 +393,13 @@
         org.slf4j
         slf4j-api
         ${slf4j.version}
+        ${hadoop.deps.scope}
       
       
         org.slf4j
         slf4j-log4j12
         ${slf4j.version}
+        ${hadoop.deps.scope}
       
       
         org.slf4j
@@ -392,6 +416,7 @@
         log4j
         log4j
         ${log4j.version}
+        ${hadoop.deps.scope}
       
       
         com.ning
@@ -401,7 +426,8 @@
       
         org.xerial.snappy
         snappy-java
-        1.1.1.6
+        ${snappy.version}
+        ${hadoop.deps.scope}
       
       
         net.jpountz.lz4
@@ -429,6 +455,7 @@
         com.google.protobuf
         protobuf-java
         ${protobuf.version}
+        ${hadoop.deps.scope}
       
       
         ${akka.group}
@@ -450,6 +477,17 @@
         akka-testkit_${scala.binary.version}
         ${akka.version}
       
+      
+        ${akka.group}
+        akka-zeromq_${scala.binary.version}
+        ${akka.version}
+        
+          
+            ${akka.group}
+            akka-actor_${scala.binary.version}
+          
+        
+      
       
         org.apache.mesos
         mesos
@@ -579,6 +617,7 @@
         org.apache.curator
         curator-recipes
         2.4.0
+        ${hadoop.deps.scope}
         
           
             org.jboss.netty
@@ -590,6 +629,7 @@
         org.apache.hadoop
         hadoop-client
         ${hadoop.version}
+        ${hadoop.deps.scope}
         
           
             asm
@@ -625,11 +665,13 @@
         org.apache.avro
         avro
         ${avro.version}
+        ${hadoop.deps.scope}
       
       
         org.apache.avro
         avro-ipc
         ${avro.version}
+        ${hadoop.deps.scope}
         
           
             io.netty
@@ -658,6 +700,7 @@
         avro-mapred
         ${avro.version}
         ${avro.mapred.classifier}
+        ${hive.deps.scope}
         
           
             io.netty
@@ -686,6 +729,7 @@
         net.java.dev.jets3t
         jets3t
         ${jets3t.version}
+        ${hadoop.deps.scope}
         
           
             commons-logging
@@ -697,6 +741,7 @@
         org.apache.hadoop
         hadoop-yarn-api
         ${yarn.version}
+        ${hadoop.deps.scope}
         
           
             javax.servlet
@@ -724,6 +769,7 @@
         org.apache.hadoop
         hadoop-yarn-common
         ${yarn.version}
+        ${hadoop.deps.scope}
         
           
             asm
@@ -780,6 +826,7 @@
         org.apache.hadoop
         hadoop-yarn-server-web-proxy
         ${yarn.version}
+        ${hadoop.deps.scope}
         
           
             asm
@@ -807,6 +854,7 @@
         org.apache.hadoop
         hadoop-yarn-client
         ${yarn.version}
+        ${hadoop.deps.scope}
         
           
             asm
@@ -831,10 +879,126 @@
         
       
       
-        
+        org.apache.zookeeper
+        zookeeper
+        ${zookeeper.version}
+        ${hadoop.deps.scope}
+      
+      
+        org.codehaus.jackson
+        jackson-core-asl
+        ${codehaus.jackson.version}
+        ${hadoop.deps.scope}
+      
+      
         org.codehaus.jackson
         jackson-mapper-asl
-        1.8.8
+        ${codehaus.jackson.version}
+        ${hadoop.deps.scope}
+      
+      
+        ${hive.group}
+        hive-beeline
+        ${hive.version}
+        ${hive.deps.scope}
+      
+      
+        ${hive.group}
+        hive-cli
+        ${hive.version}
+        ${hive.deps.scope}
+      
+      
+        ${hive.group}
+        hive-exec
+        ${hive.version}
+        ${hive.deps.scope}
+        
+          
+            commons-logging
+            commons-logging
+          
+          
+            com.esotericsoftware.kryo
+            kryo
+          
+        
+      
+      
+        ${hive.group}
+        hive-jdbc
+        ${hive.version}
+        ${hive.deps.scope}
+      
+      
+        ${hive.group}
+        hive-metastore
+        ${hive.version}
+        ${hive.deps.scope}
+      
+      
+        ${hive.group}
+        hive-serde
+        ${hive.version}
+        ${hive.deps.scope}
+        
+          
+            commons-logging
+            commons-logging
+          
+          
+            commons-logging
+            commons-logging-api
+          
+        
+      
+      
+        com.twitter
+        parquet-column
+        ${parquet.version}
+        ${parquet.deps.scope}
+      
+      
+        com.twitter
+        parquet-hadoop
+        ${parquet.version}
+        ${parquet.deps.scope}
+      
+      
+        org.apache.flume
+        flume-ng-core
+        ${flume.version}
+        ${flume.deps.scope}
+        
+          
+            io.netty
+            netty
+          
+          
+            org.apache.thrift
+            libthrift
+          
+          
+            org.mortbay.jetty
+            servlet-api
+          
+        
+      
+      
+        org.apache.flume
+        flume-ng-sdk
+        ${flume.version}
+        ${flume.deps.scope}
+        
+          
+            io.netty
+            netty
+          
+          
+            org.apache.thrift
+            libthrift
+          
+        
       
     
   
@@ -911,6 +1075,7 @@
               -Xmx1024m
               -XX:PermSize=${PermGen}
               -XX:MaxPermSize=${MaxPermGen}
+              -XX:ReservedCodeCacheSize=${CodeCacheSize}
             
             
               -source
@@ -941,32 +1106,57 @@
             true
           
         
+        
         
           org.apache.maven.plugins
           maven-surefire-plugin
-          2.17
+          2.18
+          
           
-            
-            true
+            
+              **/Test*.java
+              **/*Test.java
+              **/*TestCase.java
+              **/*Suite.java
+            
+            ${project.build.directory}/surefire-reports
+            -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m
+            
+              true
+              ${session.executionRootDirectory}
+              1
+              false
+              false
+              ${test_classpath}
+              true
+            
           
         
+        
         
           org.scalatest
           scalatest-maven-plugin
           1.0
+          
           
             ${project.build.directory}/surefire-reports
             .
             SparkTestSuite.txt
-            -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m
+            -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize}
             
+            
+              
+              ${test_classpath}
+            
             
               true
-              ${session.executionRootDirectory}
+              ${spark.test.home}
               1
               false
               false
-              ${test_classpath}
               true
             
           
@@ -989,11 +1179,6 @@
           maven-antrun-plugin
           1.7
         
-        
-          org.apache.maven.plugins
-          maven-shade-plugin
-          2.2
-        
         
           org.apache.maven.plugins
           maven-source-plugin
@@ -1022,6 +1207,9 @@
               
                 checkpoint
               
+              
+                lib_managed
+              
             
           
         
@@ -1079,6 +1267,7 @@
       
         org.apache.maven.plugins
         maven-shade-plugin
+        2.2
         
           false
           
@@ -1162,6 +1351,15 @@
           
         
       
+      
+      
+        org.apache.maven.plugins
+        maven-surefire-plugin
+      
+      
+        org.scalatest
+        scalatest-maven-plugin
+      
     
   
 
@@ -1339,53 +1537,6 @@
       
     
 
-    
-    
-      hadoop-provided
-      
-        
-          org.apache.hadoop
-          hadoop-client
-          provided
-        
-        
-          org.apache.hadoop
-          hadoop-yarn-api
-          provided
-        
-        
-          org.apache.hadoop
-          hadoop-yarn-common
-          provided
-        
-        
-          org.apache.hadoop
-          hadoop-yarn-server-web-proxy
-          provided
-        
-        
-          org.apache.hadoop
-          hadoop-yarn-client
-          provided
-        
-        
-          org.apache.avro
-          avro
-          provided
-        
-        
-          org.apache.avro
-          avro-ipc
-          provided
-        
-        
-          org.apache.zookeeper
-          zookeeper
-          ${zookeeper.version}
-          provided
-        
-      
-    
     
       hive-thriftserver
       
@@ -1438,5 +1589,25 @@
       
     
 
+    
+    
+      flume-provided
+    
+    
+      hadoop-provided
+    
+    
+      hbase-provided
+    
+    
+      hive-provided
+    
+    
+      parquet-provided
+    
   
 
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 230239aa40500..31d4c317ae569 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -53,6 +53,14 @@ object MimaExcludes {
               "org.apache.spark.mllib.linalg.Matrices.randn"),
             ProblemFilters.exclude[MissingMethodProblem](
               "org.apache.spark.mllib.linalg.Matrices.rand")
+          ) ++ Seq(
+            // SPARK-3325
+            ProblemFilters.exclude[MissingMethodProblem](
+              "org.apache.spark.streaming.api.java.JavaDStreamLike.print"),
+            // SPARK-2757
+            ProblemFilters.exclude[IncompatibleResultTypeProblem](
+              "org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler." +
+                "removeAndGetProcessor")
           )
 
         case v if v.startsWith("1.2") =>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 39ac27f820d89..46a54c6818409 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -15,6 +15,8 @@
  * limitations under the License.
  */
 
+import java.io.File
+
 import scala.util.Properties
 import scala.collection.JavaConversions._
 
@@ -23,7 +25,7 @@ import sbt.Classpaths.publishTask
 import sbt.Keys._
 import sbtunidoc.Plugin.genjavadocSettings
 import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion
-import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys}
+import com.typesafe.sbt.pom.{loadEffectivePom, PomBuild, SbtPomKeys}
 import net.virtualvoid.sbt.graph.Plugin.graphSettings
 
 object BuildCommons {
@@ -112,6 +114,17 @@ object SparkBuild extends PomBuild {
 
   override val userPropertiesMap = System.getProperties.toMap
 
+  // Handle case where hadoop.version is set via profile.
+  // Needed only because we read back this property in sbt
+  // when we create the assembly jar.
+  val pom = loadEffectivePom(new File("pom.xml"),
+    profiles = profiles,
+    userProps = userPropertiesMap)
+  if (System.getProperty("hadoop.version") == null) {
+    System.setProperty("hadoop.version",
+      pom.getProperties.get("hadoop.version").asInstanceOf[String])
+  }
+
   lazy val MavenCompile = config("m2r") extend(Compile)
   lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
 
@@ -153,7 +166,7 @@ object SparkBuild extends PomBuild {
 
   // TODO: Add Sql to mima checks
   allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl,
-    streamingFlumeSink, networkCommon, networkShuffle, networkYarn).contains(x)).foreach {
+    networkCommon, networkShuffle, networkYarn).contains(x)).foreach {
       x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)
     }
 
@@ -214,8 +227,6 @@ object OldDeps {
   def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq(
     name := "old-deps",
     scalaVersion := "2.10.4",
-    // TODO: remove this as soon as 1.2.0 is published on Maven central.
-    resolvers += "spark-staging-1038" at "https://repository.apache.org/content/repositories/orgapachespark-1038/",
     retrieveManaged := true,
     retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]",
     libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq",
@@ -256,6 +267,8 @@ object Hive {
 
   lazy val settings = Seq(
     javaOptions += "-XX:MaxPermSize=1g",
+    // Specially disable assertions since some Hive tests fail them
+    javaOptions in Test := (javaOptions in Test).value.filterNot(_ == "-ea"),
     // Multiple queries rely on the TestHive singleton. See comments there for more details.
     parallelExecution in Test := false,
     // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings
@@ -297,8 +310,7 @@ object Assembly {
         // This must match the same name used in maven (see network/yarn/pom.xml)
         "spark-" + v + "-yarn-shuffle.jar"
       } else {
-        mName + "-" + v + "-hadoop" +
-          Option(System.getProperty("hadoop.version")).getOrElse("1.0.4") + ".jar"
+        mName + "-" + v + "-hadoop" + System.getProperty("hadoop.version") + ".jar"
       }
     },
     mergeStrategy in assembly := {
@@ -385,6 +397,7 @@ object TestSettings {
     javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true",
     javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark")
       .map { case (k,v) => s"-D$k=$v" }.toSeq,
+    javaOptions in Test += "-ea",
     javaOptions in Test ++= "-Xmx3g -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g"
       .split(" ").toSeq,
     // This places test scope jars on the classpath of executors during tests.
diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst
index 5024d694b668f..f08185627d0bc 100644
--- a/python/docs/pyspark.streaming.rst
+++ b/python/docs/pyspark.streaming.rst
@@ -1,5 +1,5 @@
 pyspark.streaming module
-==================
+========================
 
 Module contents
 ---------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index ed7351d60cff2..593d74bca5fff 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -407,7 +407,7 @@ def wholeTextFiles(self, path, minPartitions=None, use_unicode=True):
 
     def binaryFiles(self, path, minPartitions=None):
         """
-        :: Experimental ::
+        .. note:: Experimental
 
         Read a directory of binary files from HDFS, a local file system
         (available on all nodes), or any Hadoop-supported file system URI
@@ -424,7 +424,7 @@ def binaryFiles(self, path, minPartitions=None):
 
     def binaryRecords(self, path, recordLength):
         """
-        :: Experimental ::
+        .. note:: Experimental
 
         Load data from a flat binary file, assuming each record is a set of numbers
         with the specified numerical format (see ByteBuffer), and the number of
diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py
index 5030a655fcbba..c3217620e3c4e 100644
--- a/python/pyspark/mllib/__init__.py
+++ b/python/pyspark/mllib/__init__.py
@@ -32,29 +32,4 @@
 import rand as random
 random.__name__ = 'random'
 random.RandomRDDs.__module__ = __name__ + '.random'
-
-
-class RandomModuleHook(object):
-    """
-    Hook to import pyspark.mllib.random
-    """
-    fullname = __name__ + '.random'
-
-    def find_module(self, name, path=None):
-        # skip all other modules
-        if not name.startswith(self.fullname):
-            return
-        return self
-
-    def load_module(self, name):
-        if name == self.fullname:
-            return random
-
-        cname = name.rsplit('.', 1)[-1]
-        try:
-            return getattr(random, cname)
-        except AttributeError:
-            raise ImportError
-
-
-sys.meta_path.append(RandomModuleHook())
+sys.modules[__name__ + '.random'] = random
diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py
index f14d0ed11cbbb..00e2e76711e84 100644
--- a/python/pyspark/mllib/classification.py
+++ b/python/pyspark/mllib/classification.py
@@ -41,7 +41,7 @@ def __init__(self, weights, intercept):
 
     def setThreshold(self, value):
         """
-        :: Experimental ::
+        .. note:: Experimental
 
         Sets the threshold that separates positive predictions from negative
         predictions. An example with prediction score greater than or equal
@@ -51,7 +51,7 @@ def setThreshold(self, value):
 
     def clearThreshold(self):
         """
-        :: Experimental ::
+        .. note:: Experimental
 
         Clears the threshold so that `predict` will output raw prediction scores.
         """
diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py
index 8cb992df2d9c7..10df6288065b8 100644
--- a/python/pyspark/mllib/feature.py
+++ b/python/pyspark/mllib/feature.py
@@ -28,7 +28,7 @@
 
 from pyspark import RDD, SparkContext
 from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper
-from pyspark.mllib.linalg import Vectors, _convert_to_vector
+from pyspark.mllib.linalg import Vectors, Vector, _convert_to_vector
 
 __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler',
            'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel']
@@ -36,7 +36,7 @@
 
 class VectorTransformer(object):
     """
-    :: DeveloperApi ::
+    .. note:: DeveloperApi
 
     Base class for transformation of a vector or RDD of vector
     """
@@ -51,12 +51,12 @@ def transform(self, vector):
 
 class Normalizer(VectorTransformer):
     """
-    :: Experimental ::
+    .. note:: Experimental
 
-    Normalizes samples individually to unit L\ :sup:`p`\ norm
+    Normalizes samples individually to unit L\ :sup:`p`\  norm
 
-    For any 1 <= `p` <= float('inf'), normalizes samples using
-    sum(abs(vector). :sup:`p`) :sup:`(1/p)` as norm.
+    For any 1 <= `p` < float('inf'), normalizes samples using
+    sum(abs(vector) :sup:`p`) :sup:`(1/p)` as norm.
 
     For `p` = float('inf'), max(abs(vector)) will be used as norm for normalization.
 
@@ -112,7 +112,7 @@ def transform(self, vector):
 
 class StandardScalerModel(JavaVectorTransformer):
     """
-    :: Experimental ::
+    .. note:: Experimental
 
     Represents a StandardScaler model that can transform vectors.
     """
@@ -129,7 +129,7 @@ def transform(self, vector):
 
 class StandardScaler(object):
     """
-    :: Experimental ::
+    .. note:: Experimental
 
     Standardizes features by removing the mean and scaling to unit
     variance using column summary statistics on the samples in the
@@ -172,7 +172,7 @@ def fit(self, dataset):
 
 class HashingTF(object):
     """
-    :: Experimental ::
+    .. note:: Experimental
 
     Maps a sequence of terms to their term frequencies using the hashing trick.
 
@@ -212,7 +212,7 @@ class IDFModel(JavaVectorTransformer):
     """
     Represents an IDF model that can transform term frequency vectors.
     """
-    def transform(self, dataset):
+    def transform(self, x):
         """
         Transforms term frequency (TF) vectors to TF-IDF vectors.
 
@@ -220,17 +220,19 @@ def transform(self, dataset):
         the terms which occur in fewer than `minDocFreq`
         documents will have an entry of 0.
 
-        :param dataset: an RDD of term frequency vectors
-        :return: an RDD of TF-IDF vectors
+        :param x: an RDD of term frequency vectors or a term frequency vector
+        :return: an RDD of TF-IDF vectors or a TF-IDF vector
         """
-        if not isinstance(dataset, RDD):
-            raise TypeError("dataset should be an RDD of term frequency vectors")
-        return JavaVectorTransformer.transform(self, dataset)
+        if isinstance(x, RDD):
+            return JavaVectorTransformer.transform(self, x)
+
+        x = _convert_to_vector(x)
+        return JavaVectorTransformer.transform(self, x)
 
 
 class IDF(object):
     """
-    :: Experimental ::
+    .. note:: Experimental
 
     Inverse document frequency (IDF).
 
@@ -255,6 +257,12 @@ class IDF(object):
     SparseVector(4, {1: 0.0, 3: 0.5754})
     DenseVector([0.0, 0.0, 1.3863, 0.863])
     SparseVector(4, {1: 0.0})
+    >>> model.transform(Vectors.dense([0.0, 1.0, 2.0, 3.0]))
+    DenseVector([0.0, 0.0, 1.3863, 0.863])
+    >>> model.transform([0.0, 1.0, 2.0, 3.0])
+    DenseVector([0.0, 0.0, 1.3863, 0.863])
+    >>> model.transform(Vectors.sparse(n, (1, 3), (1.0, 2.0)))
+    SparseVector(4, {1: 0.0, 3: 0.5754})
     """
     def __init__(self, minDocFreq=0):
         """
diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py
index f7aa2b0cb04b3..4f8491f43e457 100644
--- a/python/pyspark/mllib/linalg.py
+++ b/python/pyspark/mllib/linalg.py
@@ -178,7 +178,7 @@ def __init__(self, ar):
         elif not isinstance(ar, np.ndarray):
             ar = np.array(ar, dtype=np.float64)
         if ar.dtype != np.float64:
-            ar.astype(np.float64)
+            ar = ar.astype(np.float64)
         self.array = ar
 
     def __reduce__(self):
diff --git a/python/pyspark/mllib/rand.py b/python/pyspark/mllib/rand.py
index cb4304f92152b..20ee9d78bf5b0 100644
--- a/python/pyspark/mllib/rand.py
+++ b/python/pyspark/mllib/rand.py
@@ -99,6 +99,38 @@ def normalRDD(sc, size, numPartitions=None, seed=None):
         """
         return callMLlibFunc("normalRDD", sc._jsc, size, numPartitions, seed)
 
+    @staticmethod
+    def logNormalRDD(sc, mean, std, size, numPartitions=None, seed=None):
+        """
+        Generates an RDD comprised of i.i.d. samples from the log normal
+        distribution with the input mean and standard distribution.
+
+        :param sc: SparkContext used to create the RDD.
+        :param mean: mean for the log Normal distribution
+        :param std: std for the log Normal distribution
+        :param size: Size of the RDD.
+        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
+        :param seed: Random seed (default: a random long integer).
+        :return: RDD of float comprised of i.i.d. samples ~ log N(mean, std).
+
+        >>> from math import sqrt, exp
+        >>> mean = 0.0
+        >>> std = 1.0
+        >>> expMean = exp(mean + 0.5 * std * std)
+        >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std))
+        >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2L)
+        >>> stats = x.stats()
+        >>> stats.count()
+        1000L
+        >>> abs(stats.mean() - expMean) < 0.5
+        True
+        >>> from math import sqrt
+        >>> abs(stats.stdev() - expStd) < 0.5
+        True
+        """
+        return callMLlibFunc("logNormalRDD", sc._jsc, float(mean), float(std),
+                             size, numPartitions, seed)
+
     @staticmethod
     def poissonRDD(sc, mean, size, numPartitions=None, seed=None):
         """
@@ -125,6 +157,63 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None):
         """
         return callMLlibFunc("poissonRDD", sc._jsc, float(mean), size, numPartitions, seed)
 
+    @staticmethod
+    def exponentialRDD(sc, mean, size, numPartitions=None, seed=None):
+        """
+        Generates an RDD comprised of i.i.d. samples from the Exponential
+        distribution with the input mean.
+
+        :param sc: SparkContext used to create the RDD.
+        :param mean: Mean, or 1 / lambda, for the Exponential distribution.
+        :param size: Size of the RDD.
+        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
+        :param seed: Random seed (default: a random long integer).
+        :return: RDD of float comprised of i.i.d. samples ~ Exp(mean).
+
+        >>> mean = 2.0
+        >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2L)
+        >>> stats = x.stats()
+        >>> stats.count()
+        1000L
+        >>> abs(stats.mean() - mean) < 0.5
+        True
+        >>> from math import sqrt
+        >>> abs(stats.stdev() - sqrt(mean)) < 0.5
+        True
+        """
+        return callMLlibFunc("exponentialRDD", sc._jsc, float(mean), size, numPartitions, seed)
+
+    @staticmethod
+    def gammaRDD(sc, shape, scale, size, numPartitions=None, seed=None):
+        """
+        Generates an RDD comprised of i.i.d. samples from the Gamma
+        distribution with the input shape and scale.
+
+        :param sc: SparkContext used to create the RDD.
+        :param shape: shape (> 0) parameter for the Gamma distribution
+        :param scale: scale (> 0) parameter for the Gamma distribution
+        :param size: Size of the RDD.
+        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
+        :param seed: Random seed (default: a random long integer).
+        :return: RDD of float comprised of i.i.d. samples ~ Gamma(shape, scale).
+
+        >>> from math import sqrt
+        >>> shape = 1.0
+        >>> scale = 2.0
+        >>> expMean = shape * scale
+        >>> expStd = sqrt(shape * scale * scale)
+        >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2L)
+        >>> stats = x.stats()
+        >>> stats.count()
+        1000L
+        >>> abs(stats.mean() - expMean) < 0.5
+        True
+        >>> abs(stats.stdev() - expStd) < 0.5
+        True
+        """
+        return callMLlibFunc("gammaRDD", sc._jsc, float(shape),
+                             float(scale), size, numPartitions, seed)
+
     @staticmethod
     @toArray
     def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None):
@@ -175,6 +264,40 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None):
         """
         return callMLlibFunc("normalVectorRDD", sc._jsc, numRows, numCols, numPartitions, seed)
 
+    @staticmethod
+    @toArray
+    def logNormalVectorRDD(sc, mean, std, numRows, numCols, numPartitions=None, seed=None):
+        """
+        Generates an RDD comprised of vectors containing i.i.d. samples drawn
+        from the log normal distribution.
+
+        :param sc: SparkContext used to create the RDD.
+        :param mean: Mean of the log normal distribution
+        :param std: Standard Deviation of the log normal distribution
+        :param numRows: Number of Vectors in the RDD.
+        :param numCols: Number of elements in each Vector.
+        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
+        :param seed: Random seed (default: a random long integer).
+        :return: RDD of Vector with vectors containing i.i.d. samples ~ log `N(mean, std)`.
+
+        >>> import numpy as np
+        >>> from math import sqrt, exp
+        >>> mean = 0.0
+        >>> std = 1.0
+        >>> expMean = exp(mean + 0.5 * std * std)
+        >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std))
+        >>> mat = np.matrix(RandomRDDs.logNormalVectorRDD(sc, mean, std, \
+                               100, 100, seed=1L).collect())
+        >>> mat.shape
+        (100, 100)
+        >>> abs(mat.mean() - expMean) < 0.1
+        True
+        >>> abs(mat.std() - expStd) < 0.1
+        True
+        """
+        return callMLlibFunc("logNormalVectorRDD", sc._jsc, float(mean), float(std),
+                             numRows, numCols, numPartitions, seed)
+
     @staticmethod
     @toArray
     def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None):
@@ -205,6 +328,70 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None):
         return callMLlibFunc("poissonVectorRDD", sc._jsc, float(mean), numRows, numCols,
                              numPartitions, seed)
 
+    @staticmethod
+    @toArray
+    def exponentialVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None):
+        """
+        Generates an RDD comprised of vectors containing i.i.d. samples drawn
+        from the Exponential distribution with the input mean.
+
+        :param sc: SparkContext used to create the RDD.
+        :param mean: Mean, or 1 / lambda, for the Exponential distribution.
+        :param numRows: Number of Vectors in the RDD.
+        :param numCols: Number of elements in each Vector.
+        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`)
+        :param seed: Random seed (default: a random long integer).
+        :return: RDD of Vector with vectors containing i.i.d. samples ~ Exp(mean).
+
+        >>> import numpy as np
+        >>> mean = 0.5
+        >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1L)
+        >>> mat = np.mat(rdd.collect())
+        >>> mat.shape
+        (100, 100)
+        >>> abs(mat.mean() - mean) < 0.5
+        True
+        >>> from math import sqrt
+        >>> abs(mat.std() - sqrt(mean)) < 0.5
+        True
+        """
+        return callMLlibFunc("exponentialVectorRDD", sc._jsc, float(mean), numRows, numCols,
+                             numPartitions, seed)
+
+    @staticmethod
+    @toArray
+    def gammaVectorRDD(sc, shape, scale, numRows, numCols, numPartitions=None, seed=None):
+        """
+        Generates an RDD comprised of vectors containing i.i.d. samples drawn
+        from the Gamma distribution.
+
+        :param sc: SparkContext used to create the RDD.
+        :param shape: Shape (> 0) of the Gamma distribution
+        :param scale: Scale (> 0) of the Gamma distribution
+        :param numRows: Number of Vectors in the RDD.
+        :param numCols: Number of elements in each Vector.
+        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
+        :param seed: Random seed (default: a random long integer).
+        :return: RDD of Vector with vectors containing i.i.d. samples ~ Gamma(shape, scale).
+
+        >>> import numpy as np
+        >>> from math import sqrt
+        >>> shape = 1.0
+        >>> scale = 2.0
+        >>> expMean = shape * scale
+        >>> expStd = sqrt(shape * scale * scale)
+        >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, \
+                       100, 100, seed=1L).collect())
+        >>> mat.shape
+        (100, 100)
+        >>> abs(mat.mean() - expMean) < 0.1
+        True
+        >>> abs(mat.std() - expStd) < 0.1
+        True
+        """
+        return callMLlibFunc("gammaVectorRDD", sc._jsc, float(shape), float(scale),
+                             numRows, numCols, numPartitions, seed)
+
 
 def _test():
     import doctest
diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py
index 1980f5b03f430..c8af777a8b00d 100644
--- a/python/pyspark/mllib/stat.py
+++ b/python/pyspark/mllib/stat.py
@@ -55,7 +55,7 @@ def min(self):
 
 class ChiSqTestResult(JavaModelWrapper):
     """
-    :: Experimental ::
+    .. note:: Experimental
 
     Object containing the test results for the chi-squared hypothesis test.
     """
@@ -200,7 +200,7 @@ def corr(x, y=None, method=None):
     @staticmethod
     def chiSqTest(observed, expected=None):
         """
-        :: Experimental ::
+        .. note:: Experimental
 
         If `observed` is Vector, conduct Pearson's chi-squared goodness
         of fit test of the observed data against the expected distribution,
diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py
index 8332f8e061f48..1f48bc1219dba 100644
--- a/python/pyspark/mllib/tests.py
+++ b/python/pyspark/mllib/tests.py
@@ -23,6 +23,7 @@
 import array as pyarray
 
 from numpy import array, array_equal
+from py4j.protocol import Py4JJavaError
 
 if sys.version_info[:2] <= (2, 6):
     try:
@@ -34,7 +35,7 @@
     import unittest
 
 from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\
-    DenseMatrix
+    DenseMatrix, Vectors, Matrices
 from pyspark.mllib.regression import LabeledPoint
 from pyspark.mllib.random import RandomRDDs
 from pyspark.mllib.stat import Statistics
@@ -109,6 +110,16 @@ def test_squared_distance(self):
         self.assertEquals(0.0, _squared_distance(dv, dv))
         self.assertEquals(0.0, _squared_distance(lst, lst))
 
+    def test_conversion(self):
+        # numpy arrays should be automatically upcast to float64
+        # tests for fix of [SPARK-5089]
+        v = array([1, 2, 3, 4], dtype='float64')
+        dv = DenseVector(v)
+        self.assertTrue(dv.array.dtype == 'float64')
+        v = array([1, 2, 3, 4], dtype='float32')
+        dv = DenseVector(v)
+        self.assertTrue(dv.array.dtype == 'float64')
+
 
 class ListTests(PySparkTestCase):
 
@@ -400,6 +411,103 @@ def test_regression(self):
         self.assertTrue(dt_model.predict(features[3]) > 0)
 
 
+class ChiSqTestTests(PySparkTestCase):
+    def test_goodness_of_fit(self):
+        from numpy import inf
+
+        observed = Vectors.dense([4, 6, 5])
+        pearson = Statistics.chiSqTest(observed)
+
+        # Validated against the R command `chisq.test(c(4, 6, 5), p=c(1/3, 1/3, 1/3))`
+        self.assertEqual(pearson.statistic, 0.4)
+        self.assertEqual(pearson.degreesOfFreedom, 2)
+        self.assertAlmostEqual(pearson.pValue, 0.8187, 4)
+
+        # Different expected and observed sum
+        observed1 = Vectors.dense([21, 38, 43, 80])
+        expected1 = Vectors.dense([3, 5, 7, 20])
+        pearson1 = Statistics.chiSqTest(observed1, expected1)
+
+        # Results validated against the R command
+        # `chisq.test(c(21, 38, 43, 80), p=c(3/35, 1/7, 1/5, 4/7))`
+        self.assertAlmostEqual(pearson1.statistic, 14.1429, 4)
+        self.assertEqual(pearson1.degreesOfFreedom, 3)
+        self.assertAlmostEqual(pearson1.pValue, 0.002717, 4)
+
+        # Vectors with different sizes
+        observed3 = Vectors.dense([1.0, 2.0, 3.0])
+        expected3 = Vectors.dense([1.0, 2.0, 3.0, 4.0])
+        self.assertRaises(ValueError, Statistics.chiSqTest, observed3, expected3)
+
+        # Negative counts in observed
+        neg_obs = Vectors.dense([1.0, 2.0, 3.0, -4.0])
+        self.assertRaises(Py4JJavaError, Statistics.chiSqTest, neg_obs, expected1)
+
+        # Count = 0.0 in expected but not observed
+        zero_expected = Vectors.dense([1.0, 0.0, 3.0])
+        pearson_inf = Statistics.chiSqTest(observed, zero_expected)
+        self.assertEqual(pearson_inf.statistic, inf)
+        self.assertEqual(pearson_inf.degreesOfFreedom, 2)
+        self.assertEqual(pearson_inf.pValue, 0.0)
+
+        # 0.0 in expected and observed simultaneously
+        zero_observed = Vectors.dense([2.0, 0.0, 1.0])
+        self.assertRaises(Py4JJavaError, Statistics.chiSqTest, zero_observed, zero_expected)
+
+    def test_matrix_independence(self):
+        data = [40.0, 24.0, 29.0, 56.0, 32.0, 42.0, 31.0, 10.0, 0.0, 30.0, 15.0, 12.0]
+        chi = Statistics.chiSqTest(Matrices.dense(3, 4, data))
+
+        # Results validated against R command
+        # `chisq.test(rbind(c(40, 56, 31, 30),c(24, 32, 10, 15), c(29, 42, 0, 12)))`
+        self.assertAlmostEqual(chi.statistic, 21.9958, 4)
+        self.assertEqual(chi.degreesOfFreedom, 6)
+        self.assertAlmostEqual(chi.pValue, 0.001213, 4)
+
+        # Negative counts
+        neg_counts = Matrices.dense(2, 2, [4.0, 5.0, 3.0, -3.0])
+        self.assertRaises(Py4JJavaError, Statistics.chiSqTest, neg_counts)
+
+        # Row sum = 0.0
+        row_zero = Matrices.dense(2, 2, [0.0, 1.0, 0.0, 2.0])
+        self.assertRaises(Py4JJavaError, Statistics.chiSqTest, row_zero)
+
+        # Column sum = 0.0
+        col_zero = Matrices.dense(2, 2, [0.0, 0.0, 2.0, 2.0])
+        self.assertRaises(Py4JJavaError, Statistics.chiSqTest, col_zero)
+
+    def test_chi_sq_pearson(self):
+        data = [
+            LabeledPoint(0.0, Vectors.dense([0.5, 10.0])),
+            LabeledPoint(0.0, Vectors.dense([1.5, 20.0])),
+            LabeledPoint(1.0, Vectors.dense([1.5, 30.0])),
+            LabeledPoint(0.0, Vectors.dense([3.5, 30.0])),
+            LabeledPoint(0.0, Vectors.dense([3.5, 40.0])),
+            LabeledPoint(1.0, Vectors.dense([3.5, 40.0]))
+        ]
+
+        for numParts in [2, 4, 6, 8]:
+            chi = Statistics.chiSqTest(self.sc.parallelize(data, numParts))
+            feature1 = chi[0]
+            self.assertEqual(feature1.statistic, 0.75)
+            self.assertEqual(feature1.degreesOfFreedom, 2)
+            self.assertAlmostEqual(feature1.pValue, 0.6873, 4)
+
+            feature2 = chi[1]
+            self.assertEqual(feature2.statistic, 1.5)
+            self.assertEqual(feature2.degreesOfFreedom, 3)
+            self.assertAlmostEqual(feature2.pValue, 0.6823, 4)
+
+    def test_right_number_of_results(self):
+        num_cols = 1001
+        sparse_data = [
+            LabeledPoint(0.0, Vectors.sparse(num_cols, [(100, 2.0)])),
+            LabeledPoint(0.1, Vectors.sparse(num_cols, [(200, 1.0)]))
+        ]
+        chi = Statistics.chiSqTest(self.sc.parallelize(sparse_data))
+        self.assertEqual(len(chi), num_cols)
+        self.assertIsNotNone(chi[1000])
+
 if __name__ == "__main__":
     if not _have_scipy:
         print "NOTE: Skipping SciPy tests as it does not seem to be installed"
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 57754776faaa2..c1120cf781e5e 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -469,8 +469,7 @@ def intersection(self, other):
     def _reserialize(self, serializer=None):
         serializer = serializer or self.ctx.serializer
         if self._jrdd_deserializer != serializer:
-            if not isinstance(self, PipelinedRDD):
-                self = self.map(lambda x: x, preservesPartitioning=True)
+            self = self.map(lambda x: x, preservesPartitioning=True)
             self._jrdd_deserializer = serializer
         return self
 
@@ -1798,23 +1797,21 @@ def zip(self, other):
         def get_batch_size(ser):
             if isinstance(ser, BatchedSerializer):
                 return ser.batchSize
-            return 1
+            return 1  # not batched
 
         def batch_as(rdd, batchSize):
-            ser = rdd._jrdd_deserializer
-            if isinstance(ser, BatchedSerializer):
-                ser = ser.serializer
-            return rdd._reserialize(BatchedSerializer(ser, batchSize))
+            return rdd._reserialize(BatchedSerializer(PickleSerializer(), batchSize))
 
         my_batch = get_batch_size(self._jrdd_deserializer)
         other_batch = get_batch_size(other._jrdd_deserializer)
-        # use the smallest batchSize for both of them
-        batchSize = min(my_batch, other_batch)
-        if batchSize <= 0:
-            # auto batched or unlimited
-            batchSize = 100
-        other = batch_as(other, batchSize)
-        self = batch_as(self, batchSize)
+        if my_batch != other_batch:
+            # use the smallest batchSize for both of them
+            batchSize = min(my_batch, other_batch)
+            if batchSize <= 0:
+                # auto batched or unlimited
+                batchSize = 100
+            other = batch_as(other, batchSize)
+            self = batch_as(self, batchSize)
 
         if self.getNumPartitions() != other.getNumPartitions():
             raise ValueError("Can only zip with RDD which has the same number of partitions")
@@ -1967,7 +1964,7 @@ def _to_java_object_rdd(self):
 
     def countApprox(self, timeout, confidence=0.95):
         """
-        :: Experimental ::
+        .. note:: Experimental
         Approximate version of count() that returns a potentially incomplete
         result within a timeout, even if not all tasks have finished.
 
@@ -1980,7 +1977,7 @@ def countApprox(self, timeout, confidence=0.95):
 
     def sumApprox(self, timeout, confidence=0.95):
         """
-        :: Experimental ::
+        .. note:: Experimental
         Approximate operation to return the sum within a timeout
         or meet the confidence.
 
@@ -1996,7 +1993,7 @@ def sumApprox(self, timeout, confidence=0.95):
 
     def meanApprox(self, timeout, confidence=0.95):
         """
-        :: Experimental ::
+        .. note:: Experimental
         Approximate operation to return the mean within a timeout
         or meet the confidence.
 
@@ -2012,7 +2009,7 @@ def meanApprox(self, timeout, confidence=0.95):
 
     def countApproxDistinct(self, relativeSD=0.05):
         """
-        :: Experimental ::
+        .. note:: Experimental
         Return approximate number of distinct elements in the RDD.
 
         The algorithm used is based on streamlib's implementation of
diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py
index 33aa55f7f1429..bd08c9a6d20d6 100644
--- a/python/pyspark/serializers.py
+++ b/python/pyspark/serializers.py
@@ -463,6 +463,9 @@ def dumps(self, obj):
     def loads(self, obj):
         return self.serializer.loads(zlib.decompress(obj))
 
+    def __eq__(self, other):
+        return isinstance(other, CompressedSerializer) and self.serializer == other.serializer
+
 
 class UTF8Deserializer(Serializer):
 
@@ -489,6 +492,9 @@ def load_stream(self, stream):
         except EOFError:
             return
 
+    def __eq__(self, other):
+        return isinstance(other, UTF8Deserializer) and self.use_unicode == other.use_unicode
+
 
 def read_long(stream):
     length = stream.read(8)
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
index ae288471b0e51..0e8b398fc6b97 100644
--- a/python/pyspark/sql.py
+++ b/python/pyspark/sql.py
@@ -420,7 +420,7 @@ def fromJson(cls, json):
 
 class UserDefinedType(DataType):
     """
-    :: WARN: Spark Internal Use Only ::
+    .. note:: WARN: Spark Internal Use Only
     SQL User-Defined Type (UDT).
     """
 
@@ -788,8 +788,9 @@ def _create_converter(dataType):
         return lambda row: map(conv, row)
 
     elif isinstance(dataType, MapType):
-        conv = _create_converter(dataType.valueType)
-        return lambda row: dict((k, conv(v)) for k, v in row.iteritems())
+        kconv = _create_converter(dataType.keyType)
+        vconv = _create_converter(dataType.valueType)
+        return lambda row: dict((kconv(k), vconv(v)) for k, v in row.iteritems())
 
     elif isinstance(dataType, NullType):
         return lambda x: None
@@ -944,7 +945,7 @@ def _infer_schema_type(obj, dataType):
 
     elif isinstance(dataType, MapType):
         k, v = obj.iteritems().next()
-        return MapType(_infer_type(k),
+        return MapType(_infer_schema_type(k, dataType.keyType),
                        _infer_schema_type(v, dataType.valueType))
 
     elif isinstance(dataType, StructType):
@@ -1085,7 +1086,7 @@ def _has_struct_or_date(dt):
     elif isinstance(dt, ArrayType):
         return _has_struct_or_date(dt.elementType)
     elif isinstance(dt, MapType):
-        return _has_struct_or_date(dt.valueType)
+        return _has_struct_or_date(dt.keyType) or _has_struct_or_date(dt.valueType)
     elif isinstance(dt, DateType):
         return True
     elif isinstance(dt, UserDefinedType):
@@ -1148,12 +1149,13 @@ def List(l):
         return List
 
     elif isinstance(dataType, MapType):
-        cls = _create_cls(dataType.valueType)
+        kcls = _create_cls(dataType.keyType)
+        vcls = _create_cls(dataType.valueType)
 
         def Dict(d):
             if d is None:
                 return
-            return dict((k, _create_object(cls, v)) for k, v in d.items())
+            return dict((_create_object(kcls, k), _create_object(vcls, v)) for k, v in d.items())
 
         return Dict
 
@@ -1164,7 +1166,8 @@ def Dict(d):
         return lambda datum: dataType.deserialize(datum)
 
     elif not isinstance(dataType, StructType):
-        raise Exception("unexpected data type: %s" % dataType)
+        # no wrapper for primitive types
+        return lambda x: x
 
     class Row(tuple):
 
@@ -1668,7 +1671,7 @@ def _ssql_ctx(self):
         except Py4JError as e:
             raise Exception("You must build Spark with Hive. "
                             "Export 'SPARK_HIVE=true' and run "
-                            "sbt/sbt assembly", e)
+                            "build/sbt assembly", e)
 
     def _get_hive_ctx(self):
         return self._jvm.HiveContext(self._jsc.sc())
@@ -2082,6 +2085,34 @@ def subtract(self, other, numPartitions=None):
         else:
             raise ValueError("Can only subtract another SchemaRDD")
 
+    def sample(self, withReplacement, fraction, seed=None):
+        """
+        Return a sampled subset of this SchemaRDD.
+
+        >>> srdd = sqlCtx.inferSchema(rdd)
+        >>> srdd.sample(False, 0.5, 97).count()
+        2L
+        """
+        assert fraction >= 0.0, "Negative fraction value: %s" % fraction
+        seed = seed if seed is not None else random.randint(0, sys.maxint)
+        rdd = self._jschema_rdd.sample(withReplacement, fraction, long(seed))
+        return SchemaRDD(rdd, self.sql_ctx)
+
+    def takeSample(self, withReplacement, num, seed=None):
+        """Return a fixed-size sampled subset of this SchemaRDD.
+
+        >>> srdd = sqlCtx.inferSchema(rdd)
+        >>> srdd.takeSample(False, 2, 97)
+        [Row(field1=3, field2=u'row3'), Row(field1=1, field2=u'row1')]
+        """
+        seed = seed if seed is not None else random.randint(0, sys.maxint)
+        with SCCallSiteSync(self.context) as css:
+            bytesInJava = self._jschema_rdd.baseSchemaRDD() \
+                .takeSampleToPython(withReplacement, num, long(seed)) \
+                .iterator()
+        cls = _create_cls(self.schema())
+        return map(cls, self._collect_iterator_through_file(bytesInJava))
+
 
 def _test():
     import doctest
diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py
index 0826ddc56e844..2fe39392ff081 100644
--- a/python/pyspark/streaming/dstream.py
+++ b/python/pyspark/streaming/dstream.py
@@ -157,18 +157,20 @@ def foreachRDD(self, func):
         api = self._ssc._jvm.PythonDStream
         api.callForeachRDD(self._jdstream, jfunc)
 
-    def pprint(self):
+    def pprint(self, num=10):
         """
-        Print the first ten elements of each RDD generated in this DStream.
+        Print the first num elements of each RDD generated in this DStream.
+
+        @param num: the number of elements from the first will be printed.
         """
         def takeAndPrint(time, rdd):
-            taken = rdd.take(11)
+            taken = rdd.take(num + 1)
             print "-------------------------------------------"
             print "Time: %s" % time
             print "-------------------------------------------"
-            for record in taken[:10]:
+            for record in taken[:num]:
                 print record
-            if len(taken) > 10:
+            if len(taken) > num:
                 print "..."
             print
 
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index 32645778c2b8f..b474fcf5bfb7e 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -533,6 +533,15 @@ def test_zip_with_different_serializers(self):
         a = a._reserialize(BatchedSerializer(PickleSerializer(), 2))
         b = b._reserialize(MarshalSerializer())
         self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)])
+        # regression test for SPARK-4841
+        path = os.path.join(SPARK_HOME, "python/test_support/hello.txt")
+        t = self.sc.textFile(path)
+        cnt = t.count()
+        self.assertEqual(cnt, t.zip(t).count())
+        rdd = t.map(str)
+        self.assertEqual(cnt, t.zip(rdd).count())
+        # regression test for bug in _reserializer()
+        self.assertEqual(cnt, t.zip(rdd).count())
 
     def test_zip_with_different_number_of_items(self):
         a = self.sc.parallelize(range(5), 2)
@@ -914,6 +923,14 @@ def test_infer_schema(self):
         result = self.sqlCtx.sql("SELECT l[0].a from test2 where d['key'].d = '2'")
         self.assertEqual(1, result.first()[0])
 
+    def test_struct_in_map(self):
+        d = [Row(m={Row(i=1): Row(s="")})]
+        rdd = self.sc.parallelize(d)
+        srdd = self.sqlCtx.inferSchema(rdd)
+        k, v = srdd.first().m.items()[0]
+        self.assertEqual(1, k.i)
+        self.assertEqual("", v.s)
+
     def test_convert_row_to_dict(self):
         row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")})
         self.assertEqual(1, row.asDict()['l'][0].a)
diff --git a/repl/pom.xml b/repl/pom.xml
index 9b2290429fee5..0bc8bccf90a6d 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -68,10 +68,6 @@
       ${project.version}
       test
     
-    
-      org.eclipse.jetty
-      jetty-server
-    
     
       org.scala-lang
       scala-compiler
@@ -86,11 +82,6 @@
       org.slf4j
       jul-to-slf4j
     
-    
-      org.scalatest
-      scalatest_${scala.binary.version}
-      test
-    
     
       org.scalacheck
       scalacheck_${scala.binary.version}
@@ -108,22 +99,6 @@
           true
         
       
-      
-        org.apache.maven.plugins
-        maven-install-plugin
-        
-          true
-        
-      
-      
-        org.scalatest
-        scalatest-maven-plugin
-        
-          
-            ${basedir}/..
-          
-        
-      
       
       
         org.codehaus.mojo
diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties
index 52098993f5c3c..e7e4a4113174a 100644
--- a/repl/src/test/resources/log4j.properties
+++ b/repl/src/test/resources/log4j.properties
@@ -15,10 +15,10 @@
 # limitations under the License.
 #
 
-# Set everything to be logged to the repl/target/unit-tests.log
+# Set everything to be logged to the 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/sbt/sbt b/sbt/sbt
index 0a251d97db95c..41438251f681e 100755
--- a/sbt/sbt
+++ b/sbt/sbt
@@ -1,111 +1,29 @@
 #!/usr/bin/env bash
 
-# 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 &2
+echo "      Please update references to point to the new location." >&2
+echo "" >&2
+echo "      Invoking 'build/sbt $@' now ..." >&2
+echo "" >&2
+
+${_DIR}/../build/sbt "$@"
diff --git a/sql/README.md b/sql/README.md
index c84534da9a3d3..8d2f3cf4283e0 100644
--- a/sql/README.md
+++ b/sql/README.md
@@ -22,10 +22,10 @@ export HADOOP_HOME="/hadoop-1.0.4"
 
 Using the console
 =================
-An interactive scala console can be invoked by running `sbt/sbt hive/console`.  From here you can execute queries and inspect the various stages of query optimization.
+An interactive scala console can be invoked by running `build/sbt hive/console`.  From here you can execute queries and inspect the various stages of query optimization.
 
 ```scala
-catalyst$ sbt/sbt hive/console
+catalyst$ build/sbt hive/console
 
 [info] Starting scala interpreter...
 import org.apache.spark.sql.catalyst.analysis._
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index 1caa297e24e37..a1947fb022e54 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -50,11 +50,6 @@
       spark-core_${scala.binary.version}
       ${project.version}
     
-    
-      org.scalatest
-      scalatest_${scala.binary.version}
-      test
-    
     
       org.scalacheck
       scalacheck_${scala.binary.version}
@@ -65,11 +60,6 @@
     target/scala-${scala.binary.version}/classes
     target/scala-${scala.binary.version}/test-classes
     
-      
-        org.scalatest
-        scalatest-maven-plugin
-      
-
       
     
       org.apache.avro
       avro
-      ${avro.version}
     
     
     
       org.apache.avro
       avro-mapred
-      ${avro.version}
       ${avro.mapred.classifier}
     
-    
-      org.scalatest
-      scalatest_${scala.binary.version}
-      test
-    
     
       org.scalacheck
       scalacheck_${scala.binary.version}
@@ -161,6 +131,10 @@
       
         org.scalatest
         scalatest-maven-plugin
+        
+          
+          -da -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m
+        
       
       
          org.codehaus.mojo
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala
index 430ffb29989ea..ebf7003ff9e57 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala
@@ -21,6 +21,7 @@ import scala.language.implicitConversions
 
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.{AbstractSparkSQLParser, SqlLexical}
+import org.apache.spark.sql.hive.execution.{AddJar, AddFile, HiveNativeCommand}
 
 /**
  * A parser that recognizes all HiveQL constructs together with Spark SQL specific extensions.
@@ -52,7 +53,7 @@ private[hive] class ExtendedHiveQlParser extends AbstractSparkSQLParser {
 
   protected lazy val dfs: Parser[LogicalPlan] =
     DFS ~> wholeInput ^^ {
-      case command => NativeCommand(command.trim)
+      case command => HiveNativeCommand(command.trim)
     }
 
   private lazy val addFile: Parser[LogicalPlan] =
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 34fc21e61f60f..982e0593fcfd1 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -39,8 +39,8 @@ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperat
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.types.DecimalType
 import org.apache.spark.sql.catalyst.types.decimal.Decimal
-import org.apache.spark.sql.execution.{ExtractPythonUdfs, QueryExecutionException, Command => PhysicalCommand}
-import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand
+import org.apache.spark.sql.execution.{SparkPlan, ExecutedCommand, ExtractPythonUdfs, QueryExecutionException}
+import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DescribeHiveTableCommand}
 import org.apache.spark.sql.sources.DataSourceStrategy
 
 /**
@@ -284,7 +284,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
    * Execute the command using Hive and return the results as a sequence. Each element
    * in the sequence is one row.
    */
-  protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = {
+  protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = synchronized {
     try {
       val cmd_trimmed: String = cmd.trim()
       val tokens: Array[String] = cmd_trimmed.split("\\s+")
@@ -338,9 +338,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
   val hivePlanner = new SparkPlanner with HiveStrategies {
     val hiveContext = self
 
-    override val strategies: Seq[Strategy] = extraStrategies ++ Seq(
+    override def strategies: Seq[Strategy] = extraStrategies ++ Seq(
       DataSourceStrategy,
-      CommandStrategy(self),
+      CommandStrategy,
       HiveCommandStrategy(self),
       TakeOrdered,
       ParquetOperations,
@@ -369,11 +369,17 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
      * execution is simply passed back to Hive.
      */
     def stringResult(): Seq[String] = executedPlan match {
-      case describeHiveTableCommand: DescribeHiveTableCommand =>
+      case ExecutedCommand(desc: DescribeHiveTableCommand) =>
         // If it is a describe command for a Hive table, we want to have the output format
         // be similar with Hive.
-        describeHiveTableCommand.hiveString
-      case command: PhysicalCommand =>
+        desc.run(self).map {
+          case Row(name: String, dataType: String, comment) =>
+            Seq(name, dataType,
+              Option(comment.asInstanceOf[String]).getOrElse(""))
+              .map(s => String.format(s"%-20s", s))
+              .mkString("\t")
+        }
+      case command: ExecutedCommand =>
         command.executeCollect().map(_.head.toString)
 
       case other =>
@@ -386,7 +392,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
 
     override def simpleString: String =
       logical match {
-        case _: NativeCommand => ""
+        case _: HiveNativeCommand => ""
         case _: SetCommand => ""
         case _ => super.simpleString
       }
@@ -414,9 +420,7 @@ object HiveContext {
     case (d: Date, DateType) => new DateWritable(d).toString
     case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString
     case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8")
-    case (decimal: Decimal, DecimalType()) =>  // Hive strips trailing zeros so use its toString
-      HiveShim.createDecimal(decimal.toBigDecimal.underlying()).toString
-    case (decimal: BigDecimal, DecimalType()) =>
+    case (decimal: BigDecimal, DecimalType()) => // Hive strips trailing zeros so use its toString
       HiveShim.createDecimal(decimal.underlying()).toString
     case (other, tpe) if primitiveTypes contains tpe => other.toString
   }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index 0eeac8620f01c..a156d6f7e285d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -18,9 +18,7 @@
 package org.apache.spark.sql.hive
 
 import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar}
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory
 import org.apache.hadoop.hive.serde2.objectinspector._
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector._
 import org.apache.hadoop.hive.serde2.objectinspector.primitive._
 import org.apache.hadoop.hive.serde2.{io => hiveIo}
 import org.apache.hadoop.{io => hadoopIo}
@@ -33,6 +31,145 @@ import org.apache.spark.sql.catalyst.types.decimal.Decimal
 /* Implicit conversions */
 import scala.collection.JavaConversions._
 
+/**
+ * 1. The Underlying data type in catalyst and in Hive
+ * In catalyst:
+ *  Primitive  =>
+ *     java.lang.String
+ *     int / scala.Int
+ *     boolean / scala.Boolean
+ *     float / scala.Float
+ *     double / scala.Double
+ *     long / scala.Long
+ *     short / scala.Short
+ *     byte / scala.Byte
+ *     org.apache.spark.sql.catalyst.types.decimal.Decimal
+ *     Array[Byte]
+ *     java.sql.Date
+ *     java.sql.Timestamp
+ *  Complex Types =>
+ *    Map: scala.collection.immutable.Map
+ *    List: scala.collection.immutable.Seq
+ *    Struct:
+ *           org.apache.spark.sql.catalyst.expression.Row
+ *    Union: NOT SUPPORTED YET
+ *  The Complex types plays as a container, which can hold arbitrary data types.
+ *
+ * In Hive, the native data types are various, in UDF/UDAF/UDTF, and associated with
+ * Object Inspectors, in Hive expression evaluation framework, the underlying data are
+ * Primitive Type
+ *   Java Boxed Primitives:
+ *       org.apache.hadoop.hive.common.type.HiveVarchar
+ *       java.lang.String
+ *       java.lang.Integer
+ *       java.lang.Boolean
+ *       java.lang.Float
+ *       java.lang.Double
+ *       java.lang.Long
+ *       java.lang.Short
+ *       java.lang.Byte
+ *       org.apache.hadoop.hive.common.`type`.HiveDecimal
+ *       byte[]
+ *       java.sql.Date
+ *       java.sql.Timestamp
+ *   Writables:
+ *       org.apache.hadoop.hive.serde2.io.HiveVarcharWritable
+ *       org.apache.hadoop.io.Text
+ *       org.apache.hadoop.io.IntWritable
+ *       org.apache.hadoop.hive.serde2.io.DoubleWritable
+ *       org.apache.hadoop.io.BooleanWritable
+ *       org.apache.hadoop.io.LongWritable
+ *       org.apache.hadoop.io.FloatWritable
+ *       org.apache.hadoop.hive.serde2.io.ShortWritable
+ *       org.apache.hadoop.hive.serde2.io.ByteWritable
+ *       org.apache.hadoop.io.BytesWritable
+ *       org.apache.hadoop.hive.serde2.io.DateWritable
+ *       org.apache.hadoop.hive.serde2.io.TimestampWritable
+ *       org.apache.hadoop.hive.serde2.io.HiveDecimalWritable
+ * Complex Type
+ *   List: Object[] / java.util.List
+ *   Map: java.util.Map
+ *   Struct: Object[] / java.util.List / java POJO
+ *   Union: class StandardUnion { byte tag; Object object }
+ *
+ * NOTICE: HiveVarchar is not supported by catalyst, it will be simply considered as String type.
+ *
+ *
+ * 2. Hive ObjectInspector is a group of flexible APIs to inspect value in different data
+ *  representation, and developers can extend those API as needed, so technically,
+ *  object inspector supports arbitrary data type in java.
+ *
+ * Fortunately, only few built-in Hive Object Inspectors are used in generic udf/udaf/udtf
+ * evaluation.
+ * 1) Primitive Types (PrimitiveObjectInspector & its sub classes)
+  {{{
+   public interface PrimitiveObjectInspector {
+     // Java Primitives (java.lang.Integer, java.lang.String etc.)
+     Object getPrimitiveJavaObject(Object o);
+     // Writables (hadoop.io.IntWritable, hadoop.io.Text etc.)
+     Object getPrimitiveWritableObject(Object o);
+     // ObjectInspector only inspect the `writable` always return true, we need to check it
+     // before invoking the methods above.
+     boolean preferWritable();
+     ...
+   }
+  }}}
+
+ * 2) Complex Types:
+ *   ListObjectInspector: inspects java array or [[java.util.List]]
+ *   MapObjectInspector: inspects [[java.util.Map]]
+ *   Struct.StructObjectInspector: inspects java array, [[java.util.List]] and
+ *                                 even a normal java object (POJO)
+ *   UnionObjectInspector: (tag: Int, object data) (TODO: not supported by SparkSQL yet)
+ *
+ * 3) ConstantObjectInspector: 
+ * Constant object inspector can be either primitive type or Complex type, and it bundles a
+ * constant value as its property, usually the value is created when the constant object inspector
+ * constructed.
+ * {{{
+   public interface ConstantObjectInspector extends ObjectInspector {
+      Object getWritableConstantValue();
+      ...
+    }
+  }}}
+ * Hive provides 3 built-in constant object inspectors:
+ * Primitive Object Inspectors: 
+ *     WritableConstantStringObjectInspector
+ *     WritableConstantHiveVarcharObjectInspector
+ *     WritableConstantHiveDecimalObjectInspector
+ *     WritableConstantTimestampObjectInspector
+ *     WritableConstantIntObjectInspector
+ *     WritableConstantDoubleObjectInspector
+ *     WritableConstantBooleanObjectInspector
+ *     WritableConstantLongObjectInspector
+ *     WritableConstantFloatObjectInspector
+ *     WritableConstantShortObjectInspector
+ *     WritableConstantByteObjectInspector
+ *     WritableConstantBinaryObjectInspector
+ *     WritableConstantDateObjectInspector
+ * Map Object Inspector: 
+ *     StandardConstantMapObjectInspector
+ * List Object Inspector: 
+ *     StandardConstantListObjectInspector]]
+ * Struct Object Inspector: Hive doesn't provide the built-in constant object inspector for Struct
+ * Union Object Inspector: Hive doesn't provide the built-in constant object inspector for Union
+ *
+ *
+ * 3. This trait facilitates:
+ *    Data Unwrapping: Hive Data => Catalyst Data (unwrap)
+ *    Data Wrapping: Catalyst Data => Hive Data (wrap)
+ *    Binding the Object Inspector for Catalyst Data (toInspector)
+ *    Retrieving the Catalyst Data Type from Object Inspector (inspectorToDataType)
+ *
+ *
+ * 4. Future Improvement (TODO)
+ *   This implementation is quite ugly and inefficient:
+ *     a. Pattern matching in runtime
+ *     b. Small objects creation in catalyst data => writable
+ *     c. Unnecessary unwrap / wrap for nested UDF invoking:
+ *       e.g. date_add(printf("%s-%s-%s", a,b,c), 3)
+ *       We don't need to unwrap the data for printf and wrap it again and passes in data_add
+ */
 private[hive] trait HiveInspectors {
 
   def javaClassToDataType(clz: Class[_]): DataType = clz match {
@@ -87,10 +224,23 @@ private[hive] trait HiveInspectors {
    * @param oi   the ObjectInspector associated with the Hive Type
    * @return     convert the data into catalyst type
    * TODO return the function of (data => Any) instead for performance consideration
+   *
+   * Strictly follows the following order in unwrapping (constant OI has the higher priority):
+   *  Constant Null object inspector =>
+   *    return null
+   *  Constant object inspector =>
+   *    extract the value from constant object inspector
+   *  Check whether the `data` is null =>
+   *    return null if true
+   *  If object inspector prefers writable =>
+   *    extract writable from `data` and then get the catalyst type from the writable
+   *  Extract the java object directly from the object inspector
+   *
+   *  NOTICE: the complex data type requires recursive unwrapping.
    */
   def unwrap(data: Any, oi: ObjectInspector): Any = oi match {
-    case _ if data == null => null
-    case poi: VoidObjectInspector => null
+    case coi: ConstantObjectInspector if coi.getWritableConstantValue == null => null
+    case poi: WritableConstantStringObjectInspector => poi.getWritableConstantValue.toString
     case poi: WritableConstantHiveVarcharObjectInspector =>
       poi.getWritableConstantValue.getHiveVarchar.getValue
     case poi: WritableConstantHiveDecimalObjectInspector =>
@@ -119,12 +269,50 @@ private[hive] trait HiveInspectors {
       System.arraycopy(writable.getBytes, 0, temp, 0, temp.length)
       temp
     case poi: WritableConstantDateObjectInspector => poi.getWritableConstantValue.get()
-    case hvoi: HiveVarcharObjectInspector => hvoi.getPrimitiveJavaObject(data).getValue
-    case hdoi: HiveDecimalObjectInspector => HiveShim.toCatalystDecimal(hdoi, data)
-    // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object
-    // if next timestamp is null, so Timestamp object is cloned
-    case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone()
-    case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data)
+    case mi: StandardConstantMapObjectInspector =>
+      // take the value from the map inspector object, rather than the input data
+      mi.getWritableConstantValue.map { case (k, v) =>
+        (unwrap(k, mi.getMapKeyObjectInspector),
+          unwrap(v, mi.getMapValueObjectInspector))
+      }.toMap
+    case li: StandardConstantListObjectInspector =>
+      // take the value from the list inspector object, rather than the input data
+      li.getWritableConstantValue.map(unwrap(_, li.getListElementObjectInspector)).toSeq
+    // if the value is null, we don't care about the object inspector type
+    case _ if data == null => null
+    case poi: VoidObjectInspector => null // always be null for void object inspector
+    case pi: PrimitiveObjectInspector => pi match {
+      // We think HiveVarchar is also a String
+      case hvoi: HiveVarcharObjectInspector if hvoi.preferWritable() =>
+        hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue
+      case hvoi: HiveVarcharObjectInspector => hvoi.getPrimitiveJavaObject(data).getValue
+      case x: StringObjectInspector if x.preferWritable() =>
+        x.getPrimitiveWritableObject(data).toString
+      case x: IntObjectInspector if x.preferWritable() => x.get(data)
+      case x: BooleanObjectInspector if x.preferWritable() => x.get(data)
+      case x: FloatObjectInspector if x.preferWritable() => x.get(data)
+      case x: DoubleObjectInspector if x.preferWritable() => x.get(data)
+      case x: LongObjectInspector if x.preferWritable() => x.get(data)
+      case x: ShortObjectInspector if x.preferWritable() => x.get(data)
+      case x: ByteObjectInspector if x.preferWritable() => x.get(data)
+      case x: HiveDecimalObjectInspector => HiveShim.toCatalystDecimal(x, data)
+      case x: BinaryObjectInspector if x.preferWritable() =>
+        // BytesWritable.copyBytes() only available since Hadoop2
+        // In order to keep backward-compatible, we have to copy the
+        // bytes with old apis
+        val bw = x.getPrimitiveWritableObject(data)
+        val result = new Array[Byte](bw.getLength()) 
+        System.arraycopy(bw.getBytes(), 0, result, 0, bw.getLength())
+        result
+      case x: DateObjectInspector if x.preferWritable() =>
+        x.getPrimitiveWritableObject(data).get()
+      // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object
+      // if next timestamp is null, so Timestamp object is cloned
+      case x: TimestampObjectInspector if x.preferWritable() =>
+        x.getPrimitiveWritableObject(data).getTimestamp.clone()
+      case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone()
+      case _ => pi.getPrimitiveJavaObject(data)
+    }
     case li: ListObjectInspector =>
       Option(li.getList(data))
         .map(_.map(unwrap(_, li.getListElementObjectInspector)).toSeq)
@@ -132,10 +320,11 @@ private[hive] trait HiveInspectors {
     case mi: MapObjectInspector =>
       Option(mi.getMap(data)).map(
         _.map {
-          case (k,v) =>
+          case (k, v) =>
             (unwrap(k, mi.getMapKeyObjectInspector),
               unwrap(v, mi.getMapValueObjectInspector))
         }.toMap).orNull
+    // currently, hive doesn't provide the ConstantStructObjectInspector
     case si: StructObjectInspector =>
       val allRefs = si.getAllStructFieldRefs
       new GenericRow(
@@ -191,55 +380,89 @@ private[hive] trait HiveInspectors {
    *           the ObjectInspector should also be consistent with those returned from
    *           toInspector: DataType => ObjectInspector and
    *           toInspector: Expression => ObjectInspector
+   *
+   * Strictly follows the following order in wrapping (constant OI has the higher priority):
+   *   Constant object inspector => return the bundled value of Constant object inspector
+   *   Check whether the `a` is null => return null if true
+   *   If object inspector prefers writable object => return a Writable for the given data `a`
+   *   Map the catalyst data to the boxed java primitive
+   *
+   *  NOTICE: the complex data type requires recursive wrapping.
    */
-  def wrap(a: Any, oi: ObjectInspector): AnyRef = if (a == null) {
-    null
-  } else {
-    oi match {
-      case x: ConstantObjectInspector => x.getWritableConstantValue
-      case x: PrimitiveObjectInspector => a match {
-        // TODO what if x.preferWritable() == true? reuse the writable?
-        case s: String => s: java.lang.String
-        case i: Int => i: java.lang.Integer
-        case b: Boolean => b: java.lang.Boolean
-        case f: Float => f: java.lang.Float
-        case d: Double => d: java.lang.Double
-        case l: Long => l: java.lang.Long
-        case l: Short => l: java.lang.Short
-        case l: Byte => l: java.lang.Byte
-        case b: BigDecimal => HiveShim.createDecimal(b.underlying())
-        case d: Decimal => HiveShim.createDecimal(d.toBigDecimal.underlying())
-        case b: Array[Byte] => b
-        case d: java.sql.Date => d
-        case t: java.sql.Timestamp => t
+  def wrap(a: Any, oi: ObjectInspector): AnyRef = oi match {
+    case x: ConstantObjectInspector => x.getWritableConstantValue
+    case _ if a == null => null
+    case x: PrimitiveObjectInspector => x match {
+      // TODO we don't support the HiveVarcharObjectInspector yet.
+      case _: StringObjectInspector if x.preferWritable() => HiveShim.getStringWritable(a)
+      case _: StringObjectInspector => a.asInstanceOf[java.lang.String]
+      case _: IntObjectInspector if x.preferWritable() => HiveShim.getIntWritable(a)
+      case _: IntObjectInspector => a.asInstanceOf[java.lang.Integer]
+      case _: BooleanObjectInspector if x.preferWritable() => HiveShim.getBooleanWritable(a)
+      case _: BooleanObjectInspector => a.asInstanceOf[java.lang.Boolean]
+      case _: FloatObjectInspector if x.preferWritable() => HiveShim.getFloatWritable(a)
+      case _: FloatObjectInspector => a.asInstanceOf[java.lang.Float]
+      case _: DoubleObjectInspector if x.preferWritable() => HiveShim.getDoubleWritable(a)
+      case _: DoubleObjectInspector => a.asInstanceOf[java.lang.Double]
+      case _: LongObjectInspector if x.preferWritable() => HiveShim.getLongWritable(a)
+      case _: LongObjectInspector => a.asInstanceOf[java.lang.Long]
+      case _: ShortObjectInspector if x.preferWritable() => HiveShim.getShortWritable(a)
+      case _: ShortObjectInspector => a.asInstanceOf[java.lang.Short]
+      case _: ByteObjectInspector if x.preferWritable() => HiveShim.getByteWritable(a)
+      case _: ByteObjectInspector => a.asInstanceOf[java.lang.Byte]
+      case _: HiveDecimalObjectInspector if x.preferWritable() =>
+        HiveShim.getDecimalWritable(a.asInstanceOf[Decimal])
+      case _: HiveDecimalObjectInspector =>
+        HiveShim.createDecimal(a.asInstanceOf[Decimal].toBigDecimal.underlying())
+      case _: BinaryObjectInspector if x.preferWritable() => HiveShim.getBinaryWritable(a)
+      case _: BinaryObjectInspector => a.asInstanceOf[Array[Byte]]
+      case _: DateObjectInspector if x.preferWritable() => HiveShim.getDateWritable(a)
+      case _: DateObjectInspector => a.asInstanceOf[java.sql.Date]
+      case _: TimestampObjectInspector if x.preferWritable() => HiveShim.getTimestampWritable(a)
+      case _: TimestampObjectInspector => a.asInstanceOf[java.sql.Timestamp]
+    }
+    case x: SettableStructObjectInspector =>
+      val fieldRefs = x.getAllStructFieldRefs
+      val row = a.asInstanceOf[Seq[_]]
+      // 1. create the pojo (most likely) object
+      val result = x.create()
+      var i = 0
+      while (i < fieldRefs.length) {
+        // 2. set the property for the pojo
+        x.setStructFieldData(
+          result,
+          fieldRefs.get(i),
+          wrap(row(i), fieldRefs.get(i).getFieldObjectInspector))
+        i += 1
       }
-      case x: StructObjectInspector =>
-        val fieldRefs = x.getAllStructFieldRefs
-        val row = a.asInstanceOf[Seq[_]]
-        val result = new java.util.ArrayList[AnyRef](fieldRefs.length)
-        var i = 0
-        while (i < fieldRefs.length) {
-          result.add(wrap(row(i), fieldRefs.get(i).getFieldObjectInspector))
-          i += 1
-        }
 
-        result
-      case x: ListObjectInspector =>
-        val list = new java.util.ArrayList[Object]
-        a.asInstanceOf[Seq[_]].foreach {
-          v => list.add(wrap(v, x.getListElementObjectInspector))
-        }
-        list
-      case x: MapObjectInspector =>
-        // Some UDFs seem to assume we pass in a HashMap.
-        val hashMap = new java.util.HashMap[AnyRef, AnyRef]()
-        hashMap.putAll(a.asInstanceOf[Map[_, _]].map {
-          case (k, v) =>
-            wrap(k, x.getMapKeyObjectInspector) -> wrap(v, x.getMapValueObjectInspector)
-        })
+      result
+    case x: StructObjectInspector =>
+      val fieldRefs = x.getAllStructFieldRefs
+      val row = a.asInstanceOf[Seq[_]]
+      val result = new java.util.ArrayList[AnyRef](fieldRefs.length)
+      var i = 0
+      while (i < fieldRefs.length) {
+        result.add(wrap(row(i), fieldRefs.get(i).getFieldObjectInspector))
+        i += 1
+      }
 
-        hashMap
-    }
+      result
+    case x: ListObjectInspector =>
+      val list = new java.util.ArrayList[Object]
+      a.asInstanceOf[Seq[_]].foreach {
+        v => list.add(wrap(v, x.getListElementObjectInspector))
+      }
+      list
+    case x: MapObjectInspector =>
+      // Some UDFs seem to assume we pass in a HashMap.
+      val hashMap = new java.util.HashMap[AnyRef, AnyRef]()
+      hashMap.putAll(a.asInstanceOf[Map[_, _]].map {
+        case (k, v) =>
+          wrap(k, x.getMapKeyObjectInspector) -> wrap(v, x.getMapValueObjectInspector)
+      })
+
+      hashMap
   }
 
   def wrap(
@@ -254,6 +477,11 @@ private[hive] trait HiveInspectors {
     cache
   }
 
+  /**
+   * @param dataType Catalyst data type
+   * @return Hive java object inspector (recursively), not the Writable ObjectInspector
+   * We can easily map to the Hive built-in object inspector according to the data type.
+   */
   def toInspector(dataType: DataType): ObjectInspector = dataType match {
     case ArrayType(tpe, _) =>
       ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe))
@@ -272,12 +500,20 @@ private[hive] trait HiveInspectors {
     case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector
     case DateType => PrimitiveObjectInspectorFactory.javaDateObjectInspector
     case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector
+    // TODO decimal precision?
     case DecimalType() => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector
     case StructType(fields) =>
       ObjectInspectorFactory.getStandardStructObjectInspector(
         fields.map(f => f.name), fields.map(f => toInspector(f.dataType)))
   }
 
+  /**
+   * Map the catalyst expression to ObjectInspector, however,
+   * if the expression is [[Literal]] or foldable, a constant writable object inspector returns;
+   * Otherwise, we always get the object inspector according to its data type(in catalyst)
+   * @param expr Catalyst expression to be mapped
+   * @return Hive java objectinspector (recursively).
+   */
   def toInspector(expr: Expression): ObjectInspector = expr match {
     case Literal(value, StringType) =>
       HiveShim.getStringWritableConstantObjectInspector(value)
@@ -326,8 +562,12 @@ private[hive] trait HiveInspectors {
         })
         ObjectInspectorFactory.getStandardConstantMapObjectInspector(keyOI, valueOI, map)
       }
+    // We will enumerate all of the possible constant expressions, throw exception if we missed
     case Literal(_, dt) => sys.error(s"Hive doesn't support the constant type [$dt].")
+    // ideally, we don't test the foldable here(but in optimizer), however, some of the
+    // Hive UDF / UDAF requires its argument to be constant objectinspector, we do it eagerly.
     case _ if expr.foldable => toInspector(Literal(expr.eval(), expr.dataType))
+    // For those non constant expression, map to object inspector according to its data type
     case _ => toInspector(expr.dataType)
   }
 
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 60865638e1073..b31a3ec25096b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -20,6 +20,8 @@ package org.apache.spark.sql.hive
 import java.io.IOException
 import java.util.{List => JList}
 
+import org.apache.spark.sql.execution.SparkPlan
+
 import scala.util.parsing.combinator.RegexParsers
 
 import org.apache.hadoop.util.ReflectionUtils
@@ -261,6 +263,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
       // Wait until children are resolved.
       case p: LogicalPlan if !p.childrenResolved => p
 
+      // TODO extra is in type of ASTNode which means the logical plan is not resolved
+      // Need to think about how to implement the CreateTableAsSelect.resolved
       case CreateTableAsSelect(db, tableName, child, allowExisting, Some(extra: ASTNode)) =>
         val (dbName, tblName) = processDatabaseAndTableName(db, tableName)
         val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase)
@@ -284,7 +288,24 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
           Some(sa.getQB().getTableDesc)
         }
 
-        CreateTableAsSelect(Some(databaseName), tblName, child, allowExisting, desc)
+        execution.CreateTableAsSelect(
+          databaseName,
+          tableName,
+          child,
+          allowExisting,
+          desc)
+
+      case p: LogicalPlan if p.resolved => p
+
+      case p @ CreateTableAsSelect(db, tableName, child, allowExisting, None) =>
+        val (dbName, tblName) = processDatabaseAndTableName(db, tableName)
+        val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase)
+        execution.CreateTableAsSelect(
+          databaseName,
+          tableName,
+          child,
+          allowExisting,
+          None)
     }
   }
 
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index cd4e5a239ec66..8a9613cf96e54 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable}
 
 /* Implicit conversions */
 import scala.collection.JavaConversions._
@@ -44,14 +45,6 @@ import scala.collection.JavaConversions._
  */
 private[hive] case object NativePlaceholder extends Command
 
-private[hive] case class AddFile(filePath: String) extends Command
-
-private[hive] case class AddJar(path: String) extends Command
-
-private[hive] case class DropTable(tableName: String, ifExists: Boolean) extends Command
-
-private[hive] case class AnalyzeTable(tableName: String) extends Command
-
 /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */
 private[hive] object HiveQl {
   protected val nativeCommands = Seq(
@@ -239,10 +232,10 @@ private[hive] object HiveQl {
     try {
       val tree = getAst(sql)
       if (nativeCommands contains tree.getText) {
-        NativeCommand(sql)
+        HiveNativeCommand(sql)
       } else {
         nodeToPlan(tree) match {
-          case NativePlaceholder => NativeCommand(sql)
+          case NativePlaceholder => HiveNativeCommand(sql)
           case other => other
         }
       }
@@ -393,6 +386,42 @@ private[hive] object HiveQl {
     (db, tableName)
   }
 
+  /**
+   * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) 
+   * is equivalent to 
+   * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2
+   * Check the following link for details.
+   * 
+https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup
+   *
+   * The bitmask denotes the grouping expressions validity for a grouping set,
+   * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive)
+   * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of 
+   * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively.
+   */
+  protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = {
+    val (keyASTs, setASTs) = children.partition( n => n match {
+        case Token("TOK_GROUPING_SETS_EXPRESSION", children) => false // grouping sets
+        case _ => true // grouping keys
+      })
+
+    val keys = keyASTs.map(nodeToExpr).toSeq
+    val keyMap = keyASTs.map(_.toStringTree).zipWithIndex.toMap
+
+    val bitmasks: Seq[Int] = setASTs.map(set => set match {
+      case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0
+      case Token("TOK_GROUPING_SETS_EXPRESSION", children) => 
+        children.foldLeft(0)((bitmap, col) => {
+          val colString = col.asInstanceOf[ASTNode].toStringTree()
+          require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list")
+          bitmap | 1 << keyMap(colString)
+        })
+      case _ => sys.error("Expect GROUPING SETS clause")
+    })
+
+    (keys, bitmasks)
+  }
+
   protected def nodeToPlan(node: Node): LogicalPlan = node match {
     // Special drop table that also uncaches.
     case Token("TOK_DROPTABLE",
@@ -520,6 +549,9 @@ private[hive] object HiveQl {
             selectDistinctClause ::
             whereClause ::
             groupByClause ::
+            rollupGroupByClause ::
+            cubeGroupByClause ::
+            groupingSetsClause ::
             orderByClause ::
             havingClause ::
             sortByClause ::
@@ -535,6 +567,9 @@ private[hive] object HiveQl {
               "TOK_SELECTDI",
               "TOK_WHERE",
               "TOK_GROUPBY",
+              "TOK_ROLLUP_GROUPBY",
+              "TOK_CUBE_GROUPBY",
+              "TOK_GROUPING_SETS",
               "TOK_ORDERBY",
               "TOK_HAVING",
               "TOK_SORTBY",
@@ -603,16 +638,33 @@ private[hive] object HiveQl {
 
         // The projection of the query can either be a normal projection, an aggregation
         // (if there is a group by) or a script transformation.
-        val withProject = transformation.getOrElse {
-          // Not a transformation so must be either project or aggregation.
-          val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr))
-
-          groupByClause match {
-            case Some(groupBy) =>
-              Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withLateralView)
-            case None =>
-              Project(selectExpressions, withLateralView)
-          }
+        val withProject: LogicalPlan = transformation.getOrElse {
+          val selectExpressions = 
+            nameExpressions(select.getChildren.flatMap(selExprNodeToExpr).toSeq)
+          Seq(
+            groupByClause.map(e => e match {
+              case Token("TOK_GROUPBY", children) =>
+                // Not a transformation so must be either project or aggregation.
+                Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView)
+              case _ => sys.error("Expect GROUP BY")
+            }),
+            groupingSetsClause.map(e => e match {
+              case Token("TOK_GROUPING_SETS", children) =>
+                val(groupByExprs, masks) = extractGroupingSet(children)
+                GroupingSets(masks, groupByExprs, withLateralView, selectExpressions)
+              case _ => sys.error("Expect GROUPING SETS")
+            }),
+            rollupGroupByClause.map(e => e match {
+              case Token("TOK_ROLLUP_GROUPBY", children) =>
+                Rollup(children.map(nodeToExpr), withLateralView, selectExpressions)
+              case _ => sys.error("Expect WITH ROLLUP")
+            }),
+            cubeGroupByClause.map(e => e match {
+              case Token("TOK_CUBE_GROUPBY", children) =>
+                Cube(children.map(nodeToExpr), withLateralView, selectExpressions)
+              case _ => sys.error("Expect WITH CUBE")
+            }), 
+            Some(Project(selectExpressions, withLateralView))).flatten.head
         }
 
         val withDistinct =
@@ -628,16 +680,16 @@ private[hive] object HiveQl {
         val withSort =
           (orderByClause, sortByClause, distributeByClause, clusterByClause) match {
             case (Some(totalOrdering), None, None, None) =>
-              Sort(totalOrdering.getChildren.map(nodeToSortOrder), withHaving)
+              Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withHaving)
             case (None, Some(perPartitionOrdering), None, None) =>
-              SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), withHaving)
+              Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withHaving)
             case (None, None, Some(partitionExprs), None) =>
               Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving)
             case (None, Some(perPartitionOrdering), Some(partitionExprs), None) =>
-              SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder),
+              Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false,
                 Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving))
             case (None, None, None, Some(clusterExprs)) =>
-              SortPartitions(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)),
+              Sort(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), false,
                 Repartition(clusterExprs.getChildren.map(nodeToExpr), withHaving))
             case (None, None, None, None) => withHaving
             case _ => sys.error("Unsupported set of ordering / distribution clauses.")
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index 5f02e95ac3c34..d3f6381b69a4d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -17,9 +17,6 @@
 
 package org.apache.spark.sql.hive
 
-import org.apache.hadoop.hive.ql.parse.ASTNode
-import org.apache.hadoop.hive.ql.plan.CreateTableDesc
-
 import org.apache.spark.annotation.Experimental
 import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
 import org.apache.spark.sql.catalyst.expressions._
@@ -28,7 +25,7 @@ import org.apache.spark.sql.catalyst.planning._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.types.StringType
-import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan, PhysicalRDD}
+import org.apache.spark.sql.execution._
 import org.apache.spark.sql.hive
 import org.apache.spark.sql.hive.execution._
 import org.apache.spark.sql.parquet.ParquetRelation
@@ -177,25 +174,10 @@ private[hive] trait HiveStrategies {
     def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
       case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) =>
         execution.InsertIntoHiveTable(
-          table, partition, planLater(child), overwrite)(hiveContext) :: Nil
+          table, partition, planLater(child), overwrite) :: Nil
       case hive.InsertIntoHiveTable(table: MetastoreRelation, partition, child, overwrite) =>
         execution.InsertIntoHiveTable(
-          table, partition, planLater(child), overwrite)(hiveContext) :: Nil
-      case logical.CreateTableAsSelect(
-             Some(database), tableName, child, allowExisting, Some(desc: CreateTableDesc)) =>
-        execution.CreateTableAsSelect(
-          database,
-          tableName,
-          child,
-          allowExisting,
-          Some(desc)) :: Nil
-      case logical.CreateTableAsSelect(Some(database), tableName, child, allowExisting, None) =>
-        execution.CreateTableAsSelect(
-          database,
-          tableName,
-          child,
-          allowExisting,
-          None) :: Nil
+          table, partition, planLater(child), overwrite) :: Nil
       case _ => Nil
     }
   }
@@ -210,8 +192,9 @@ private[hive] trait HiveStrategies {
         // Filter out all predicates that only deal with partition keys, these are given to the
         // hive table scan operator to be used for partition pruning.
         val partitionKeyIds = AttributeSet(relation.partitionKeys)
-        val (pruningPredicates, otherPredicates) = predicates.partition {
-          _.references.subsetOf(partitionKeyIds)
+        val (pruningPredicates, otherPredicates) = predicates.partition { predicate =>
+          !predicate.references.isEmpty &&
+          predicate.references.subsetOf(partitionKeyIds)
         }
 
         pruneFilterProject(
@@ -226,23 +209,14 @@ private[hive] trait HiveStrategies {
 
   case class HiveCommandStrategy(context: HiveContext) extends Strategy {
     def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
-      case logical.NativeCommand(sql) => NativeCommand(sql, plan.output)(context) :: Nil
-
-      case hive.DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil
-
-      case hive.AddJar(path) => execution.AddJar(path) :: Nil
-
-      case hive.AddFile(path) => execution.AddFile(path) :: Nil
-
-      case hive.AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil
-
       case describe: logical.DescribeCommand =>
         val resolvedTable = context.executePlan(describe.table).analyzed
         resolvedTable match {
           case t: MetastoreRelation =>
-            Seq(DescribeHiveTableCommand(t, describe.output, describe.isExtended)(context))
+            ExecutedCommand(
+              DescribeHiveTableCommand(t, describe.output, describe.isExtended)) :: Nil
           case o: LogicalPlan =>
-            Seq(DescribeCommand(planLater(o), describe.output)(context))
+            ExecutedCommand(DescribeCommand(planLater(o), describe.output)) :: Nil
         }
 
       case _ => Nil
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index f60bc3788e3e4..c368715f7c6f5 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -57,10 +57,15 @@ class HadoopTableReader(
     @transient hiveExtraConf: HiveConf)
   extends TableReader {
 
-  // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless
-  // it is smaller than what Spark suggests.
-  private val _minSplitsPerRDD = math.max(
-    sc.hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinPartitions)
+  // Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local".
+  // https://hadoop.apache.org/docs/r1.0.4/mapred-default.html
+  //
+  // In order keep consistency with Hive, we will let it be 0 in local mode also.
+  private val _minSplitsPerRDD = if (sc.sparkContext.isLocal) {
+    0 // will splitted based on block by default.
+  } else {
+    math.max(sc.hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinPartitions)
+  }
 
   // TODO: set aws s3 credentials.
 
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index bb79ad5538046..b2149bd95a336 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -20,9 +20,6 @@ package org.apache.spark.sql.hive.test
 import java.io.File
 import java.util.{Set => JavaSet}
 
-import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.ql.session.SessionState
-
 import scala.collection.mutable
 import scala.language.implicitConversions
 
@@ -37,10 +34,11 @@ import org.apache.hadoop.hive.serde2.avro.AvroSerDe
 import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.spark.util.Utils
 import org.apache.spark.sql.catalyst.analysis._
-import org.apache.spark.sql.catalyst.plans.logical.{CacheTableCommand, LogicalPlan, NativeCommand}
+import org.apache.spark.sql.catalyst.plans.logical.{CacheTableCommand, LogicalPlan}
 import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.SQLConf
+import org.apache.spark.sql.hive.execution.HiveNativeCommand
 
 /* Implicit conversions */
 import scala.collection.JavaConversions._
@@ -161,7 +159,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
   abstract class QueryExecution extends super.QueryExecution {
     override lazy val analyzed = {
       val describedTables = logical match {
-        case NativeCommand(describedTable(tbl)) => tbl :: Nil
+        case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil
         case CacheTableCommand(tbl, _, _) => tbl :: Nil
         case _ => Nil
       }
@@ -426,6 +424,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
       // other sql exec here.
       runSqlHive("RESET")
       // For some reason, RESET does not reset the following variables...
+      // https://issues.apache.org/jira/browse/HIVE-9004
+      runSqlHive("set hive.table.parameters.default=")
       runSqlHive("set datanucleus.cache.collections=true")
       runSqlHive("set datanucleus.cache.collections.lazy=true")
       // Lots of tests fail if we do not change the partition whitelist from the default.
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
index b83689ceabb84..fe21454e7fb38 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
@@ -20,10 +20,10 @@ package org.apache.spark.sql.hive.execution
 import org.apache.hadoop.hive.ql.plan.CreateTableDesc
 
 import org.apache.spark.annotation.Experimental
-import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.catalyst.expressions.Row
 import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan}
-import org.apache.spark.sql.execution.{SparkPlan, Command, LeafNode}
+import org.apache.spark.sql.execution.RunnableCommand
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.hive.MetastoreRelation
 
@@ -44,28 +44,23 @@ case class CreateTableAsSelect(
     tableName: String,
     query: LogicalPlan,
     allowExisting: Boolean,
-    desc: Option[CreateTableDesc]) extends LeafNode with Command {
+    desc: Option[CreateTableDesc]) extends RunnableCommand {
 
-  def output = Seq.empty
+  override def run(sqlContext: SQLContext) = {
+    val hiveContext = sqlContext.asInstanceOf[HiveContext]
+    lazy val metastoreRelation: MetastoreRelation = {
+      // Create Hive Table
+      hiveContext.catalog.createTable(database, tableName, query.output, allowExisting, desc)
 
-  private[this] def sc = sqlContext.asInstanceOf[HiveContext]
-
-  // A lazy computing of the metastoreRelation
-  private[this] lazy val metastoreRelation: MetastoreRelation = {
-    // Create Hive Table
-    sc.catalog.createTable(database, tableName, query.output, allowExisting, desc)
-
-    // Get the Metastore Relation
-    sc.catalog.lookupRelation(Some(database), tableName, None) match {
-      case r: MetastoreRelation => r
+      // Get the Metastore Relation
+      hiveContext.catalog.lookupRelation(Some(database), tableName, None) match {
+        case r: MetastoreRelation => r
+      }
     }
-  }
-
-  override protected[sql] lazy val sideEffectResult: Seq[Row] = {
     // TODO ideally, we should get the output data ready first and then
     // add the relation into catalog, just in case of failure occurs while data
     // processing.
-    if (sc.catalog.tableExists(Some(database), tableName)) {
+    if (hiveContext.catalog.tableExists(Some(database), tableName)) {
       if (allowExisting) {
         // table already exists, will do nothing, to keep consistent with Hive
       } else {
@@ -73,17 +68,12 @@ case class CreateTableAsSelect(
           new org.apache.hadoop.hive.metastore.api.AlreadyExistsException(s"$database.$tableName")
       }
     } else {
-      sc.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd
+      hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd
     }
 
     Seq.empty[Row]
   }
 
-  override def execute(): RDD[Row] = {
-    sideEffectResult
-    sparkContext.emptyRDD[Row]
-  }
-
   override def argString: String = {
     s"[Database:$database, TableName: $tableName, InsertIntoHiveTable]\n" + query.toString
   }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
index 5d98834c6fb33..bfacc51ef57ab 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
@@ -22,11 +22,11 @@ import scala.collection.JavaConversions._
 import org.apache.hadoop.hive.metastore.api.FieldSchema
 
 import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
-import org.apache.spark.sql.execution.{Command, LeafNode}
+import org.apache.spark.sql.execution.{SparkPlan, RunnableCommand}
 import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation}
 import org.apache.spark.sql.hive.HiveShim
+import org.apache.spark.sql.SQLContext
 
 /**
  * Implementation for "describe [extended] table".
@@ -36,21 +36,10 @@ import org.apache.spark.sql.hive.HiveShim
 @DeveloperApi
 case class DescribeHiveTableCommand(
     table: MetastoreRelation,
-    output: Seq[Attribute],
-    isExtended: Boolean)(
-    @transient context: HiveContext)
-  extends LeafNode with Command {
+    override val output: Seq[Attribute],
+    isExtended: Boolean) extends RunnableCommand {
 
-  // Strings with the format like Hive. It is used for result comparison in our unit tests.
-  lazy val hiveString: Seq[String] = sideEffectResult.map {
-    case Row(name: String, dataType: String, comment) =>
-      Seq(name, dataType,
-        Option(comment.asInstanceOf[String]).getOrElse(""))
-        .map(s => String.format(s"%-20s", s))
-        .mkString("\t")
-  }
-
-  override protected lazy val sideEffectResult: Seq[Row] = {
+  override def run(sqlContext: SQLContext) = {
     // Trying to mimic the format of Hive's output. But not exactly the same.
     var results: Seq[(String, String, String)] = Nil
 
@@ -75,6 +64,4 @@ case class DescribeHiveTableCommand(
       Row(name, dataType, comment)
     }
   }
-
-  override def otherCopyArgs = context :: Nil
 }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
similarity index 64%
rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala
rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
index 6930c2babd117..8ba818af5f9d0 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
@@ -1,38 +1,38 @@
-/*
- * 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.sql.hive.execution
-
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow, Row}
-import org.apache.spark.sql.execution.{Command, LeafNode}
-import org.apache.spark.sql.hive.HiveContext
-
-/**
- * :: DeveloperApi ::
- */
-@DeveloperApi
-case class NativeCommand(
-    sql: String, output: Seq[Attribute])(
-    @transient context: HiveContext)
-  extends LeafNode with Command {
-
-  override protected lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_))
-
-  override def otherCopyArgs = context :: Nil
-}
+/*
+ * 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.sql.hive.execution
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row}
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.hive.HiveContext
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.types.StringType
+
+/**
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
+case class HiveNativeCommand(sql: String) extends RunnableCommand {
+
+  override def output =
+    Seq(AttributeReference("result", StringType, nullable = false)())
+
+  override def run(sqlContext: SQLContext) =
+    sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_))
+}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index 81390f626726c..ca0ec1513917f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -21,7 +21,6 @@ import java.util
 
 import scala.collection.JavaConversions._
 
-import org.apache.hadoop.hive.common.`type`.HiveVarchar
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars
 import org.apache.hadoop.hive.metastore.MetaStoreUtils
@@ -31,14 +30,12 @@ import org.apache.hadoop.hive.ql.{Context, ErrorMsg}
 import org.apache.hadoop.hive.serde2.Serializer
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
 import org.apache.hadoop.hive.serde2.objectinspector._
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.{JavaHiveDecimalObjectInspector, JavaHiveVarcharObjectInspector}
 import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf}
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-import org.apache.spark.sql.execution.{Command, SparkPlan, UnaryNode}
+import org.apache.spark.sql.execution.{UnaryNode, SparkPlan}
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc}
 import org.apache.spark.sql.hive.HiveShim._
@@ -52,10 +49,9 @@ case class InsertIntoHiveTable(
     table: MetastoreRelation,
     partition: Map[String, Option[String]],
     child: SparkPlan,
-    overwrite: Boolean)
-    (@transient sc: HiveContext)
-  extends UnaryNode with Command with HiveInspectors {
+    overwrite: Boolean) extends UnaryNode with HiveInspectors {
 
+  @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext]
   @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass
   @transient private lazy val hiveContext = new Context(sc.hiveconf)
   @transient private lazy val db = Hive.get(sc.hiveconf)
@@ -66,8 +62,6 @@ case class InsertIntoHiveTable(
     serializer
   }
 
-  override def otherCopyArgs = sc :: Nil
-
   def output = child.output
 
   def saveAsHiveFile(
@@ -134,7 +128,7 @@ case class InsertIntoHiveTable(
    *
    * Note: this is run once and then kept to avoid double insertions.
    */
-  override protected[sql] lazy val sideEffectResult: Seq[Row] = {
+  protected[sql] lazy val sideEffectResult: Seq[Row] = {
     // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer
     // instances within the closure, since Serializer is not serializable while TableDesc is.
     val tableDesc = table.tableDesc
@@ -256,4 +250,8 @@ case class InsertIntoHiveTable(
     // TODO: implement hive compatibility as rules.
     Seq.empty[Row]
   }
+
+  override def executeCollect(): Array[Row] = sideEffectResult.toArray
+
+  override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1)
 }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
index 903075edf7e04..6fc4153f6a5df 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
@@ -18,10 +18,10 @@
 package org.apache.spark.sql.hive.execution
 
 import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.sql.execution.{Command, LeafNode}
+import org.apache.spark.sql.execution.RunnableCommand
 import org.apache.spark.sql.hive.HiveContext
+import org.apache.spark.sql.SQLContext
 
 /**
  * :: DeveloperApi ::
@@ -32,13 +32,10 @@ import org.apache.spark.sql.hive.HiveContext
  * in the Hive metastore.
  */
 @DeveloperApi
-case class AnalyzeTable(tableName: String) extends LeafNode with Command {
-  def hiveContext = sqlContext.asInstanceOf[HiveContext]
+case class AnalyzeTable(tableName: String) extends RunnableCommand {
 
-  def output = Seq.empty
-
-  override protected lazy val sideEffectResult: Seq[Row] = {
-    hiveContext.analyze(tableName)
+  override def run(sqlContext: SQLContext) = {
+    sqlContext.asInstanceOf[HiveContext].analyze(tableName)
     Seq.empty[Row]
   }
 }
@@ -48,12 +45,12 @@ case class AnalyzeTable(tableName: String) extends LeafNode with Command {
  * Drops a table from the metastore and removes it if it is cached.
  */
 @DeveloperApi
-case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with Command {
-  def hiveContext = sqlContext.asInstanceOf[HiveContext]
-
-  def output = Seq.empty
+case class DropTable(
+    tableName: String,
+    ifExists: Boolean) extends RunnableCommand {
 
-  override protected lazy val sideEffectResult: Seq[Row] = {
+  override def run(sqlContext: SQLContext) = {
+    val hiveContext = sqlContext.asInstanceOf[HiveContext]
     val ifExistsClause = if (ifExists) "IF EXISTS " else ""
     hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName")
     hiveContext.catalog.unregisterTable(None, tableName)
@@ -65,12 +62,10 @@ case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with
  * :: DeveloperApi ::
  */
 @DeveloperApi
-case class AddJar(path: String) extends LeafNode with Command {
-  def hiveContext = sqlContext.asInstanceOf[HiveContext]
+case class AddJar(path: String) extends RunnableCommand {
 
-  override def output = Seq.empty
-
-  override protected lazy val sideEffectResult: Seq[Row] = {
+  override def run(sqlContext: SQLContext) = {
+    val hiveContext = sqlContext.asInstanceOf[HiveContext]
     hiveContext.runSqlHive(s"ADD JAR $path")
     hiveContext.sparkContext.addJar(path)
     Seq.empty[Row]
@@ -81,12 +76,10 @@ case class AddJar(path: String) extends LeafNode with Command {
  * :: DeveloperApi ::
  */
 @DeveloperApi
-case class AddFile(path: String) extends LeafNode with Command {
-  def hiveContext = sqlContext.asInstanceOf[HiveContext]
-
-  override def output = Seq.empty
+case class AddFile(path: String) extends RunnableCommand {
 
-  override protected lazy val sideEffectResult: Seq[Row] = {
+  override def run(sqlContext: SQLContext) = {
+    val hiveContext = sqlContext.asInstanceOf[HiveContext]
     hiveContext.runSqlHive(s"ADD FILE $path")
     hiveContext.sparkContext.addFile(path)
     Seq.empty[Row]
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index ed2e96df8ad77..7d863f9d89dae 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -166,6 +166,7 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr
 
   override def eval(input: Row): Any = {
     returnInspector // Make sure initialized.
+
     var i = 0
     while (i < children.length) {
       val idx = i
@@ -193,12 +194,13 @@ private[hive] case class HiveGenericUdaf(
 
   @transient
   protected lazy val objectInspector  = {
-    resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray)
+    val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors.toArray, false, false)
+    resolver.getEvaluator(parameterInfo)
       .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray)
   }
 
   @transient
-  protected lazy val inspectors = children.map(_.dataType).map(toInspector)
+  protected lazy val inspectors = children.map(toInspector)
 
   def dataType: DataType = inspectorToDataType(objectInspector)
 
@@ -223,12 +225,13 @@ private[hive] case class HiveUdaf(
 
   @transient
   protected lazy val objectInspector  = {
-    resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray)
+    val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors.toArray, false, false)
+    resolver.getEvaluator(parameterInfo)
       .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray)
   }
 
   @transient
-  protected lazy val inspectors = children.map(_.dataType).map(toInspector)
+  protected lazy val inspectors = children.map(toInspector)
 
   def dataType: DataType = inspectorToDataType(objectInspector)
 
@@ -261,7 +264,7 @@ private[hive] case class HiveGenericUdtf(
   protected lazy val function: GenericUDTF = funcWrapper.createFunction()
 
   @transient
-  protected lazy val inputInspectors = children.map(_.dataType).map(toInspector)
+  protected lazy val inputInspectors = children.map(toInspector)
 
   @transient
   protected lazy val outputInspector = function.initialize(inputInspectors.toArray)
@@ -334,10 +337,13 @@ private[hive] case class HiveUdafFunction(
     } else {
       funcWrapper.createFunction[AbstractGenericUDAFResolver]()
     }
-
-  private val inspectors = exprs.map(_.dataType).map(toInspector).toArray
-
-  private val function = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray)
+  
+  private val inspectors = exprs.map(toInspector).toArray
+    
+  private val function = { 
+    val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors, false, false)
+    resolver.getEvaluator(parameterInfo) 
+  }
 
   private val returnInspector = function.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors)
 
@@ -350,9 +356,12 @@ private[hive] case class HiveUdafFunction(
   @transient
   val inputProjection = new InterpretedProjection(exprs)
 
+  @transient
+  protected lazy val cached = new Array[AnyRef](exprs.length)
+  
   def update(input: Row): Unit = {
     val inputs = inputProjection(input).asInstanceOf[Seq[AnyRef]].toArray
-    function.iterate(buffer, inputs)
+    function.iterate(buffer, wrap(inputs, inspectors, cached))
   }
 }
 
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala
index abed299cd957f..2a16c9d1a27c9 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala
@@ -32,7 +32,7 @@ import org.apache.hadoop.io.Writable
  * when "spark.sql.hive.convertMetastoreParquet" is set to true.
  */
 @deprecated("No code should depend on FakeParquetHiveSerDe as it is only intended as a " +
-            "placeholder in the Hive MetaStore")
+            "placeholder in the Hive MetaStore", "1.2.0")
 class FakeParquetSerDe extends SerDe {
   override def getObjectInspector: ObjectInspector = new ObjectInspector {
     override def getCategory: Category = Category.PRIMITIVE
diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java
index d2d39a8c4dc28..808e2986d3b77 100644
--- a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java
+++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java
@@ -23,25 +23,21 @@
 
 public class UDFListListInt extends UDF {
   /**
-   *
    * @param obj
-   *   SQL schema: array>
-   *   Java Type: List>
-   * @return
+   *   SQL schema: array<struct<x: int, y: int, z: int>>
+   *   Java Type: List<List<Integer>>
    */
+  @SuppressWarnings("unchecked")
   public long evaluate(Object obj) {
     if (obj == null) {
-      return 0l;
+      return 0L;
     }
-    List listList = (List) obj;
+    List> listList = (List>) obj;
     long retVal = 0;
-    for (List aList : listList) {
-      @SuppressWarnings("unchecked")
-      List list = (List) aList;
-      @SuppressWarnings("unchecked")
-      Integer someInt = (Integer) list.get(1);
+    for (List aList : listList) {
+      Number someInt = (Number) aList.get(1);
       try {
-        retVal += (long) (someInt.intValue());
+        retVal += someInt.longValue();
       } catch (NullPointerException e) {
         System.out.println(e);
       }
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-0-432a3ade72afd99cfb4b886692c15e55 b/sql/hive/src/test/resources/golden/create_like_tbl_props-0-432a3ade72afd99cfb4b886692c15e55
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-1-f8481dcbc8f2731bab8ac3894511ff9f b/sql/hive/src/test/resources/golden/create_like_tbl_props-1-f8481dcbc8f2731bab8ac3894511ff9f
new file mode 100644
index 0000000000000..7d8744672aa11
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-1-f8481dcbc8f2731bab8ac3894511ff9f
@@ -0,0 +1,28 @@
+# col_name            	data_type           	comment             
+	 	 
+key                 	int                 	                    
+value               	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+Owner:              	hcheng              	 
+CreateTime:         	Fri Nov 28 00:04:15 PST 2014	 
+LastAccessTime:     	UNKNOWN             	 
+Protect Mode:       	None                	 
+Retention:          	0                   	 
+Location:           	file:/tmp/sparkHiveWarehouse3490012261419180285/test_table	 
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	key                 	value               
+	transient_lastDdlTime	1417161855          
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-10-2a2d6f2c92e32285dd4c4dd3d0faa9 b/sql/hive/src/test/resources/golden/create_like_tbl_props-10-2a2d6f2c92e32285dd4c4dd3d0faa9
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-11-b4f47dcb46073bda6fb1d9f96e8b36e6 b/sql/hive/src/test/resources/golden/create_like_tbl_props-11-b4f47dcb46073bda6fb1d9f96e8b36e6
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-12-184ab0f730b53d1b8b4f4e1feade9824 b/sql/hive/src/test/resources/golden/create_like_tbl_props-12-184ab0f730b53d1b8b4f4e1feade9824
new file mode 100644
index 0000000000000..b55c2dcfe4934
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-12-184ab0f730b53d1b8b4f4e1feade9824
@@ -0,0 +1,29 @@
+# col_name            	data_type           	comment             
+	 	 
+key                 	int                 	                    
+value               	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+Owner:              	hcheng              	 
+CreateTime:         	Fri Nov 28 00:04:16 PST 2014	 
+LastAccessTime:     	UNKNOWN             	 
+Protect Mode:       	None                	 
+Retention:          	0                   	 
+Location:           	file:/tmp/sparkHiveWarehouse3490012261419180285/test_table4	 
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	key                 	value               
+	key1                	value1              
+	transient_lastDdlTime	1417161856          
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-2-62c728aff7df8cd2bd2c114c9076a1ff b/sql/hive/src/test/resources/golden/create_like_tbl_props-2-62c728aff7df8cd2bd2c114c9076a1ff
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-2-62c728aff7df8cd2bd2c114c9076a1ff
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-3-3320f357baaadfe13820349b8d941865 b/sql/hive/src/test/resources/golden/create_like_tbl_props-3-3320f357baaadfe13820349b8d941865
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-4-f59c262efb0482b555ae867abef4040f b/sql/hive/src/test/resources/golden/create_like_tbl_props-4-f59c262efb0482b555ae867abef4040f
new file mode 100644
index 0000000000000..8c94e5fb15561
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-4-f59c262efb0482b555ae867abef4040f
@@ -0,0 +1,28 @@
+# col_name            	data_type           	comment             
+	 	 
+key                 	int                 	                    
+value               	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+Owner:              	hcheng              	 
+CreateTime:         	Fri Nov 28 00:04:15 PST 2014	 
+LastAccessTime:     	UNKNOWN             	 
+Protect Mode:       	None                	 
+Retention:          	0                   	 
+Location:           	file:/tmp/sparkHiveWarehouse3490012261419180285/test_table1	 
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	key1                	value1              
+	transient_lastDdlTime	1417161855          
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-5-11e1ff4a0cf4ee27f5ccb5f267643cfd b/sql/hive/src/test/resources/golden/create_like_tbl_props-5-11e1ff4a0cf4ee27f5ccb5f267643cfd
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-6-c49698cf69779ee8a519e2566c6b2acb b/sql/hive/src/test/resources/golden/create_like_tbl_props-6-c49698cf69779ee8a519e2566c6b2acb
new file mode 100644
index 0000000000000..ddec982d168c0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-6-c49698cf69779ee8a519e2566c6b2acb
@@ -0,0 +1,29 @@
+# col_name            	data_type           	comment             
+	 	 
+key                 	int                 	                    
+value               	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+Owner:              	hcheng              	 
+CreateTime:         	Fri Nov 28 00:04:16 PST 2014	 
+LastAccessTime:     	UNKNOWN             	 
+Protect Mode:       	None                	 
+Retention:          	0                   	 
+Location:           	file:/tmp/sparkHiveWarehouse3490012261419180285/test_table2	 
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	key1                	value1              
+	key2                	value2              
+	transient_lastDdlTime	1417161856          
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-7-25f0c8b81d949d73737ee3a5398fc9f7 b/sql/hive/src/test/resources/golden/create_like_tbl_props-7-25f0c8b81d949d73737ee3a5398fc9f7
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-7-25f0c8b81d949d73737ee3a5398fc9f7
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-8-69b6bc0b259beb299874e7cdfc5edb1b b/sql/hive/src/test/resources/golden/create_like_tbl_props-8-69b6bc0b259beb299874e7cdfc5edb1b
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-9-9461431e44ae60a529cc309d8f325dbc b/sql/hive/src/test/resources/golden/create_like_tbl_props-9-9461431e44ae60a529cc309d8f325dbc
new file mode 100644
index 0000000000000..547d4fbdf34d3
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-9-9461431e44ae60a529cc309d8f325dbc
@@ -0,0 +1,29 @@
+# col_name            	data_type           	comment             
+	 	 
+key                 	int                 	                    
+value               	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+Owner:              	hcheng              	 
+CreateTime:         	Fri Nov 28 00:04:16 PST 2014	 
+LastAccessTime:     	UNKNOWN             	 
+Protect Mode:       	None                	 
+Retention:          	0                   	 
+Location:           	file:/tmp/sparkHiveWarehouse3490012261419180285/test_table3	 
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	key1                	value1              
+	key2                	value3              
+	transient_lastDdlTime	1417161856          
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-0-dc7fc9ce5109ef459ee84ccfbb12d2c0 b/sql/hive/src/test/resources/golden/create_view_translate-0-dc7fc9ce5109ef459ee84ccfbb12d2c0
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-1-3896ae0e680a5fdc01833533b11c07bb b/sql/hive/src/test/resources/golden/create_view_translate-1-3896ae0e680a5fdc01833533b11c07bb
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-10-7016e1e3a4248564f3d08cddad7ae116 b/sql/hive/src/test/resources/golden/create_view_translate-10-7016e1e3a4248564f3d08cddad7ae116
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-11-e27c6a59a833dcbc2e5cdb7ff7972828 b/sql/hive/src/test/resources/golden/create_view_translate-11-e27c6a59a833dcbc2e5cdb7ff7972828
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-2-6b4caec6d7e3a91e61720bbd6b7697f0 b/sql/hive/src/test/resources/golden/create_view_translate-2-6b4caec6d7e3a91e61720bbd6b7697f0
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-3-30dc3e80e3873af5115e4f5e39078a13 b/sql/hive/src/test/resources/golden/create_view_translate-3-30dc3e80e3873af5115e4f5e39078a13
new file mode 100644
index 0000000000000..cec5f77033aa4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/create_view_translate-3-30dc3e80e3873af5115e4f5e39078a13
@@ -0,0 +1,27 @@
+# col_name            	data_type           	comment             
+	 	 
+key                 	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+Owner:              	animal              	 
+CreateTime:         	Mon Dec 29 00:57:55 PST 2014	 
+LastAccessTime:     	UNKNOWN             	 
+Protect Mode:       	None                	 
+Retention:          	0                   	 
+Table Type:         	VIRTUAL_VIEW        	 
+Table Parameters:	 	 
+	transient_lastDdlTime	1419843475          
+	 	 
+# Storage Information	 	 
+SerDe Library:      	null                	 
+InputFormat:        	org.apache.hadoop.mapred.SequenceFileInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+	 	 
+# View Information	 	 
+View Original Text: 	select cast(key as string) from src	 
+View Expanded Text: 	select cast(`src`.`key` as string) from `default`.`src`	 
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-4-cefb7530126f9e60cb4a29441d578f23 b/sql/hive/src/test/resources/golden/create_view_translate-4-cefb7530126f9e60cb4a29441d578f23
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-5-856ea995681b18a543dc0e53b8b43a8e b/sql/hive/src/test/resources/golden/create_view_translate-5-856ea995681b18a543dc0e53b8b43a8e
new file mode 100644
index 0000000000000..bf582fc0964a3
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/create_view_translate-5-856ea995681b18a543dc0e53b8b43a8e
@@ -0,0 +1,32 @@
+# col_name            	data_type           	comment             
+	 	 
+key                 	int                 	                    
+value               	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+Owner:              	animal              	 
+CreateTime:         	Mon Dec 29 00:57:55 PST 2014	 
+LastAccessTime:     	UNKNOWN             	 
+Protect Mode:       	None                	 
+Retention:          	0                   	 
+Table Type:         	VIRTUAL_VIEW        	 
+Table Parameters:	 	 
+	transient_lastDdlTime	1419843475          
+	 	 
+# Storage Information	 	 
+SerDe Library:      	null                	 
+InputFormat:        	org.apache.hadoop.mapred.SequenceFileInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+	 	 
+# View Information	 	 
+View Original Text: 	select key, value from (	 
+  select key, value from src	 	 
+) a	 	 
+View Expanded Text: 	select key, value from (	 
+  select `src`.`key`, `src`.`value` from `default`.`src`	 	 
+) `a`	 	 
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-6-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/create_view_translate-6-a14cfe3eff322066e61023ec06c7735d
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-7-e947bf2dacc907825df154a4131a3fcc b/sql/hive/src/test/resources/golden/create_view_translate-7-e947bf2dacc907825df154a4131a3fcc
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-8-b1a99b0beffb0b298aec9233ecc0707f b/sql/hive/src/test/resources/golden/create_view_translate-8-b1a99b0beffb0b298aec9233ecc0707f
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_view_translate-9-fc0dc39c4796d917685e0797bc4a9786 b/sql/hive/src/test/resources/golden/create_view_translate-9-fc0dc39c4796d917685e0797bc4a9786
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/decimal_1-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/decimal_1-0-50131c0ba7b7a6b65c789a5a8497bada
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-0-50131c0ba7b7a6b65c789a5a8497bada
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 b/sql/hive/src/test/resources/golden/decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b b/sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b
new file mode 100644
index 0000000000000..98d9bcb75a685
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b
@@ -0,0 +1 @@
+17
diff --git a/sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba b/sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba
new file mode 100644
index 0000000000000..53aca7545dac7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba
@@ -0,0 +1 @@
+17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8 b/sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8
new file mode 100644
index 0000000000000..53aca7545dac7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8
@@ -0,0 +1 @@
+17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 b/sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3
new file mode 100644
index 0000000000000..53aca7545dac7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3
@@ -0,0 +1 @@
+17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 b/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0
new file mode 100644
index 0000000000000..c4a17c1b14c88
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0
@@ -0,0 +1 @@
+1969-12-31 16:00:17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/decimal_1-2-ee665100ca5de3a006df43e97cfa707 b/sql/hive/src/test/resources/golden/decimal_1-2-ee665100ca5de3a006df43e97cfa707
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/decimal_1-3-80fc87cab17ceffea334afbb230a6653 b/sql/hive/src/test/resources/golden/decimal_1-3-80fc87cab17ceffea334afbb230a6653
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119 b/sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119
new file mode 100644
index 0000000000000..711809abcc925
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119
@@ -0,0 +1,3 @@
+t                   	decimal(4,2)        	                    
+u                   	decimal(5,0)        	                    
+v                   	decimal(10,0)       	                    
diff --git a/sql/hive/src/test/resources/golden/decimal_1-5-bfab296ca5693e647e33899dfeeb256 b/sql/hive/src/test/resources/golden/decimal_1-5-bfab296ca5693e647e33899dfeeb256
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5 b/sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5
new file mode 100644
index 0000000000000..27ba77ddaf615
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5
@@ -0,0 +1 @@
+true
diff --git a/sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 b/sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1
new file mode 100644
index 0000000000000..98d9bcb75a685
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1
@@ -0,0 +1 @@
+17
diff --git a/sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8 b/sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8
new file mode 100644
index 0000000000000..98d9bcb75a685
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8
@@ -0,0 +1 @@
+17
diff --git a/sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde b/sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde
new file mode 100644
index 0000000000000..98d9bcb75a685
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde
@@ -0,0 +1 @@
+17
diff --git a/sql/hive/src/test/resources/golden/empty aggregate input-0-bbd21aa0c1faf4c1fe6d8a822b416349 b/sql/hive/src/test/resources/golden/empty aggregate input-0-bbd21aa0c1faf4c1fe6d8a822b416349
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/empty aggregate input-0-bbd21aa0c1faf4c1fe6d8a822b416349	
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/file_split_for_small_table-0-7a45831bf96814d9a7fc3d78fb7bd8dc b/sql/hive/src/test/resources/golden/file_split_for_small_table-0-7a45831bf96814d9a7fc3d78fb7bd8dc
new file mode 100644
index 0000000000000..b70e127e82d05
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/file_split_for_small_table-0-7a45831bf96814d9a7fc3d78fb7bd8dc
@@ -0,0 +1,500 @@
+0	val_0
+0	val_0
+0	val_0
+2	val_2
+4	val_4
+5	val_5
+5	val_5
+5	val_5
+8	val_8
+9	val_9
+10	val_10
+11	val_11
+12	val_12
+12	val_12
+15	val_15
+15	val_15
+17	val_17
+18	val_18
+18	val_18
+19	val_19
+20	val_20
+24	val_24
+24	val_24
+26	val_26
+26	val_26
+27	val_27
+28	val_28
+30	val_30
+33	val_33
+34	val_34
+35	val_35
+35	val_35
+35	val_35
+37	val_37
+37	val_37
+41	val_41
+42	val_42
+42	val_42
+43	val_43
+44	val_44
+47	val_47
+51	val_51
+51	val_51
+53	val_53
+54	val_54
+57	val_57
+58	val_58
+58	val_58
+64	val_64
+65	val_65
+66	val_66
+67	val_67
+67	val_67
+69	val_69
+70	val_70
+70	val_70
+70	val_70
+72	val_72
+72	val_72
+74	val_74
+76	val_76
+76	val_76
+77	val_77
+78	val_78
+80	val_80
+82	val_82
+83	val_83
+83	val_83
+84	val_84
+84	val_84
+85	val_85
+86	val_86
+87	val_87
+90	val_90
+90	val_90
+90	val_90
+92	val_92
+95	val_95
+95	val_95
+96	val_96
+97	val_97
+97	val_97
+98	val_98
+98	val_98
+100	val_100
+100	val_100
+103	val_103
+103	val_103
+104	val_104
+104	val_104
+105	val_105
+111	val_111
+113	val_113
+113	val_113
+114	val_114
+116	val_116
+118	val_118
+118	val_118
+119	val_119
+119	val_119
+119	val_119
+120	val_120
+120	val_120
+125	val_125
+125	val_125
+126	val_126
+128	val_128
+128	val_128
+128	val_128
+129	val_129
+129	val_129
+131	val_131
+133	val_133
+134	val_134
+134	val_134
+136	val_136
+137	val_137
+137	val_137
+138	val_138
+138	val_138
+138	val_138
+138	val_138
+143	val_143
+145	val_145
+146	val_146
+146	val_146
+149	val_149
+149	val_149
+150	val_150
+152	val_152
+152	val_152
+153	val_153
+155	val_155
+156	val_156
+157	val_157
+158	val_158
+160	val_160
+162	val_162
+163	val_163
+164	val_164
+164	val_164
+165	val_165
+165	val_165
+166	val_166
+167	val_167
+167	val_167
+167	val_167
+168	val_168
+169	val_169
+169	val_169
+169	val_169
+169	val_169
+170	val_170
+172	val_172
+172	val_172
+174	val_174
+174	val_174
+175	val_175
+175	val_175
+176	val_176
+176	val_176
+177	val_177
+178	val_178
+179	val_179
+179	val_179
+180	val_180
+181	val_181
+183	val_183
+186	val_186
+187	val_187
+187	val_187
+187	val_187
+189	val_189
+190	val_190
+191	val_191
+191	val_191
+192	val_192
+193	val_193
+193	val_193
+193	val_193
+194	val_194
+195	val_195
+195	val_195
+196	val_196
+197	val_197
+197	val_197
+199	val_199
+199	val_199
+199	val_199
+200	val_200
+200	val_200
+201	val_201
+202	val_202
+203	val_203
+203	val_203
+205	val_205
+205	val_205
+207	val_207
+207	val_207
+208	val_208
+208	val_208
+208	val_208
+209	val_209
+209	val_209
+213	val_213
+213	val_213
+214	val_214
+216	val_216
+216	val_216
+217	val_217
+217	val_217
+218	val_218
+219	val_219
+219	val_219
+221	val_221
+221	val_221
+222	val_222
+223	val_223
+223	val_223
+224	val_224
+224	val_224
+226	val_226
+228	val_228
+229	val_229
+229	val_229
+230	val_230
+230	val_230
+230	val_230
+230	val_230
+230	val_230
+233	val_233
+233	val_233
+235	val_235
+237	val_237
+237	val_237
+238	val_238
+238	val_238
+239	val_239
+239	val_239
+241	val_241
+242	val_242
+242	val_242
+244	val_244
+247	val_247
+248	val_248
+249	val_249
+252	val_252
+255	val_255
+255	val_255
+256	val_256
+256	val_256
+257	val_257
+258	val_258
+260	val_260
+262	val_262
+263	val_263
+265	val_265
+265	val_265
+266	val_266
+272	val_272
+272	val_272
+273	val_273
+273	val_273
+273	val_273
+274	val_274
+275	val_275
+277	val_277
+277	val_277
+277	val_277
+277	val_277
+278	val_278
+278	val_278
+280	val_280
+280	val_280
+281	val_281
+281	val_281
+282	val_282
+282	val_282
+283	val_283
+284	val_284
+285	val_285
+286	val_286
+287	val_287
+288	val_288
+288	val_288
+289	val_289
+291	val_291
+292	val_292
+296	val_296
+298	val_298
+298	val_298
+298	val_298
+302	val_302
+305	val_305
+306	val_306
+307	val_307
+307	val_307
+308	val_308
+309	val_309
+309	val_309
+310	val_310
+311	val_311
+311	val_311
+311	val_311
+315	val_315
+316	val_316
+316	val_316
+316	val_316
+317	val_317
+317	val_317
+318	val_318
+318	val_318
+318	val_318
+321	val_321
+321	val_321
+322	val_322
+322	val_322
+323	val_323
+325	val_325
+325	val_325
+327	val_327
+327	val_327
+327	val_327
+331	val_331
+331	val_331
+332	val_332
+333	val_333
+333	val_333
+335	val_335
+336	val_336
+338	val_338
+339	val_339
+341	val_341
+342	val_342
+342	val_342
+344	val_344
+344	val_344
+345	val_345
+348	val_348
+348	val_348
+348	val_348
+348	val_348
+348	val_348
+351	val_351
+353	val_353
+353	val_353
+356	val_356
+360	val_360
+362	val_362
+364	val_364
+365	val_365
+366	val_366
+367	val_367
+367	val_367
+368	val_368
+369	val_369
+369	val_369
+369	val_369
+373	val_373
+374	val_374
+375	val_375
+377	val_377
+378	val_378
+379	val_379
+382	val_382
+382	val_382
+384	val_384
+384	val_384
+384	val_384
+386	val_386
+389	val_389
+392	val_392
+393	val_393
+394	val_394
+395	val_395
+395	val_395
+396	val_396
+396	val_396
+396	val_396
+397	val_397
+397	val_397
+399	val_399
+399	val_399
+400	val_400
+401	val_401
+401	val_401
+401	val_401
+401	val_401
+401	val_401
+402	val_402
+403	val_403
+403	val_403
+403	val_403
+404	val_404
+404	val_404
+406	val_406
+406	val_406
+406	val_406
+406	val_406
+407	val_407
+409	val_409
+409	val_409
+409	val_409
+411	val_411
+413	val_413
+413	val_413
+414	val_414
+414	val_414
+417	val_417
+417	val_417
+417	val_417
+418	val_418
+419	val_419
+421	val_421
+424	val_424
+424	val_424
+427	val_427
+429	val_429
+429	val_429
+430	val_430
+430	val_430
+430	val_430
+431	val_431
+431	val_431
+431	val_431
+432	val_432
+435	val_435
+436	val_436
+437	val_437
+438	val_438
+438	val_438
+438	val_438
+439	val_439
+439	val_439
+443	val_443
+444	val_444
+446	val_446
+448	val_448
+449	val_449
+452	val_452
+453	val_453
+454	val_454
+454	val_454
+454	val_454
+455	val_455
+457	val_457
+458	val_458
+458	val_458
+459	val_459
+459	val_459
+460	val_460
+462	val_462
+462	val_462
+463	val_463
+463	val_463
+466	val_466
+466	val_466
+466	val_466
+467	val_467
+468	val_468
+468	val_468
+468	val_468
+468	val_468
+469	val_469
+469	val_469
+469	val_469
+469	val_469
+469	val_469
+470	val_470
+472	val_472
+475	val_475
+477	val_477
+478	val_478
+478	val_478
+479	val_479
+480	val_480
+480	val_480
+480	val_480
+481	val_481
+482	val_482
+483	val_483
+484	val_484
+485	val_485
+487	val_487
+489	val_489
+489	val_489
+489	val_489
+489	val_489
+490	val_490
+491	val_491
+492	val_492
+492	val_492
+493	val_493
+494	val_494
+495	val_495
+496	val_496
+497	val_497
+498	val_498
+498	val_498
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-0-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/groupby_grouping_id1-0-caf1c5fd299fdbdb655234d01d44caf2
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-c0f14def6a135cc50cba364e810ce28e b/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-c0f14def6a135cc50cba364e810ce28e
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa b/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa
new file mode 100644
index 0000000000000..76280c6f3a1c8
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa
@@ -0,0 +1,18 @@
+NULL	NULL	0
+NULL	11	2
+NULL	12	2
+NULL	13	2
+NULL	17	2
+NULL	18	2
+NULL	28	2
+1	NULL	1
+1	11	3
+2	NULL	1
+2	12	3
+3	NULL	1
+3	13	3
+7	NULL	1
+7	17	3
+8	NULL	1
+8	18	3
+8	28	3
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51 b/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51
new file mode 100644
index 0000000000000..b18af4e5dd637
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51
@@ -0,0 +1,12 @@
+0	NULL	NULL
+1	1	NULL
+3	1	11
+1	2	NULL
+3	2	12
+1	3	NULL
+3	3	13
+1	7	NULL
+3	7	17
+1	8	NULL
+3	8	18
+3	8	28
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391 b/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391
new file mode 100644
index 0000000000000..5a7ac193cb11b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391
@@ -0,0 +1,18 @@
+NULL	NULL	0	0
+NULL	11	2	2
+NULL	12	2	2
+NULL	13	2	2
+NULL	17	2	2
+NULL	18	2	2
+NULL	28	2	2
+1	NULL	1	1
+1	11	3	3
+2	NULL	1	1
+2	12	3	3
+3	NULL	1	1
+3	13	3	3
+7	NULL	1	1
+7	17	3	3
+8	NULL	1	1
+8	18	3	3
+8	28	3	3
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-0-20539c642c514a590caca2f11395007e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-0-20539c642c514a590caca2f11395007e
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d1f3215e349f056c8de60b87a6a9855e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d1f3215e349f056c8de60b87a6a9855e
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586
new file mode 100644
index 0000000000000..66ac2d65ce245
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586
@@ -0,0 +1,11 @@
+NULL	NULL	0	6
+1	NULL	1	2
+1	NULL	3	1
+1	1	3	1
+2	NULL	1	1
+2	2	3	1
+3	NULL	1	2
+3	NULL	3	1
+3	3	3	1
+4	NULL	1	1
+4	5	3	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74
new file mode 100644
index 0000000000000..8c1e9630ebfd7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74
@@ -0,0 +1,3 @@
+0	1
+1	4
+3	6
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82
new file mode 100644
index 0000000000000..42fc2290c2cba
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82
@@ -0,0 +1,53 @@
+0	0
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586
new file mode 100644
index 0000000000000..66ac2d65ce245
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586
@@ -0,0 +1,11 @@
+NULL	NULL	0	6
+1	NULL	1	2
+1	NULL	3	1
+1	1	3	1
+2	NULL	1	1
+2	2	3	1
+3	NULL	1	2
+3	NULL	3	1
+3	3	3	1
+4	NULL	1	1
+4	5	3	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74
new file mode 100644
index 0000000000000..8c1e9630ebfd7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74
@@ -0,0 +1,3 @@
+0	1
+1	4
+3	6
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82
new file mode 100644
index 0000000000000..42fc2290c2cba
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82
@@ -0,0 +1,53 @@
+0	0
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-0-91128722f50ec00b51e0bf6fe5695cd1 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-0-91128722f50ec00b51e0bf6fe5695cd1
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-3673d61d0944adeba77438d882839de4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a
new file mode 100644
index 0000000000000..7967c04c92149
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a
@@ -0,0 +1,6 @@
+8	1	1
+5	2	2
+1	1	3
+2	2	4
+2	3	5
+3	2	8
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a
new file mode 100644
index 0000000000000..da4a754efa0e0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c
new file mode 100644
index 0000000000000..da4a754efa0e0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62
new file mode 100644
index 0000000000000..2d1b73d564955
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62
@@ -0,0 +1,11 @@
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542
new file mode 100644
index 0000000000000..fe6c4d001180a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542
@@ -0,0 +1,14 @@
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+1
+2
+3
+5
+8
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423
new file mode 100644
index 0000000000000..24d5fc2851703
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423
@@ -0,0 +1,5 @@
+1
+2
+3
+5
+8
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324
new file mode 100644
index 0000000000000..8ba0d6d25a6f0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324
@@ -0,0 +1,5 @@
+2.0	1
+4.0	1
+5.0	2
+7.0	1
+9.0	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-1-91128722f50ec00b51e0bf6fe5695cd1 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-1-91128722f50ec00b51e0bf6fe5695cd1
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3
new file mode 100644
index 0000000000000..cabc9bb1d918f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3
@@ -0,0 +1,15 @@
+NULL	NULL	46
+NULL	1	8
+NULL	2	28
+NULL	3	10
+1	NULL	6
+1	1	6
+2	NULL	18
+2	2	8
+2	3	10
+3	NULL	16
+3	2	16
+5	NULL	4
+5	2	4
+8	NULL	2
+8	1	2
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-3673d61d0944adeba77438d882839de4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-3-59adefab34d80e8e185b2ad03877d381 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-3-59adefab34d80e8e185b2ad03877d381
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a
new file mode 100644
index 0000000000000..da4a754efa0e0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-5-1163c486fd7e2c4346805fb035e2f268 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-5-1163c486fd7e2c4346805fb035e2f268
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc
new file mode 100644
index 0000000000000..b20db4c79aa70
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc
@@ -0,0 +1,15 @@
+NULL	NULL	23.0
+NULL	1	4.0
+NULL	2	14.0
+NULL	3	5.0
+1	NULL	3.0
+1	1	3.0
+2	NULL	9.0
+2	2	4.0
+2	3	5.0
+3	NULL	8.0
+3	2	8.0
+5	NULL	2.0
+5	2	2.0
+8	NULL	1.0
+8	1	1.0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-7-39db88427f92cb770b6daa38610c04e7 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-7-39db88427f92cb770b6daa38610c04e7
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-8-e464ec3d5461bda47eac3d1ef8617786 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-8-e464ec3d5461bda47eac3d1ef8617786
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-9-74126b100714164d13cbb3bff436c2ff b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-9-74126b100714164d13cbb3bff436c2ff
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-38373b67d392924967a4695689d2164e b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-38373b67d392924967a4695689d2164e
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-e8964b2aaeb388064c9fdac5ec687824 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-e8964b2aaeb388064c9fdac5ec687824
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-c1942a377b1a440d4ed3dd05fed445d b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-c1942a377b1a440d4ed3dd05fed445d
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-d95cf9dfae402d369f338b8516845e02 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-d95cf9dfae402d369f338b8516845e02
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-5-6623f95d90d929a6d2c8171a0698d4fd b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-5-6623f95d90d929a6d2c8171a0698d4fd
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef
new file mode 100644
index 0000000000000..b2d08949e9795
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef
@@ -0,0 +1,16 @@
+NULL	NULL	3.8333333333333335	12
+NULL	1	2.0	5
+NULL	2	5.2	5
+NULL	3	5.0	2
+1	NULL	2.6666666666666665	3
+1	1	3.0	2
+1	2	2.0	1
+2	NULL	5.2	5
+2	2	5.333333333333333	3
+2	3	5.0	2
+3	NULL	8.0	1
+3	2	8.0	1
+5	NULL	2.0	1
+5	1	2.0	1
+8	NULL	1.0	2
+8	1	1.0	2
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-8-f32c4a191759237733a10cd721b49966 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-8-f32c4a191759237733a10cd721b49966
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef
new file mode 100644
index 0000000000000..b2d08949e9795
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef
@@ -0,0 +1,16 @@
+NULL	NULL	3.8333333333333335	12
+NULL	1	2.0	5
+NULL	2	5.2	5
+NULL	3	5.0	2
+1	NULL	2.6666666666666665	3
+1	1	3.0	2
+1	2	2.0	1
+2	NULL	5.2	5
+2	2	5.333333333333333	3
+2	3	5.0	2
+3	NULL	8.0	1
+3	2	8.0	1
+5	NULL	2.0	1
+5	1	2.0	1
+8	NULL	1.0	2
+8	1	1.0	2
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-2-1789808269c8bd0f6259227f07da1a6a b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-2-1789808269c8bd0f6259227f07da1a6a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-3673d61d0944adeba77438d882839de4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-4-b51d813d0c5a410d8cf9765d85005a01 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-4-b51d813d0c5a410d8cf9765d85005a01
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438
new file mode 100644
index 0000000000000..33060f0d51729
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438
@@ -0,0 +1,13 @@
+1	NULL	1	1	NULL	1
+1	NULL	1	1	1	1
+1	1	1	1	NULL	1
+1	1	1	1	1	1
+2	NULL	2	2	NULL	2
+2	NULL	2	2	2	1
+2	NULL	2	2	3	1
+2	2	1	2	NULL	2
+2	2	1	2	2	1
+2	2	1	2	3	1
+2	3	1	2	NULL	2
+2	3	1	2	2	1
+2	3	1	2	3	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-7-98f7522ce136cdffb0c14163e613c250 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-7-98f7522ce136cdffb0c14163e613c250
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438
new file mode 100644
index 0000000000000..33060f0d51729
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438
@@ -0,0 +1,13 @@
+1	NULL	1	1	NULL	1
+1	NULL	1	1	1	1
+1	1	1	1	NULL	1
+1	1	1	1	1	1
+2	NULL	2	2	NULL	2
+2	NULL	2	2	2	1
+2	NULL	2	2	3	1
+2	2	1	2	NULL	2
+2	2	1	2	2	1
+2	2	1	2	3	1
+2	3	1	2	NULL	2
+2	3	1	2	2	1
+2	3	1	2	3	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-2-1789808269c8bd0f6259227f07da1a6a b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-2-1789808269c8bd0f6259227f07da1a6a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-3673d61d0944adeba77438d882839de4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f
new file mode 100644
index 0000000000000..da4a754efa0e0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-7-6544a382d851f916616c4386fdcf0ed8 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-7-6544a382d851f916616c4386fdcf0ed8
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f
new file mode 100644
index 0000000000000..da4a754efa0e0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1
diff --git a/sql/hive/src/test/resources/golden/udf5-0-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf5-0-2a18d9570d9b676e240cda76df818c42
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/udf5-1-f60851dc36f579e83d6848d7d3c589e6 b/sql/hive/src/test/resources/golden/udf5-1-f60851dc36f579e83d6848d7d3c589e6
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/udf5-2-e08fad5ccbf165f44ecabb9356e58b24 b/sql/hive/src/test/resources/golden/udf5-2-e08fad5ccbf165f44ecabb9356e58b24
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/udf5-3-d23017942dc49be9f5a61430201371bf b/sql/hive/src/test/resources/golden/udf5-3-d23017942dc49be9f5a61430201371bf
new file mode 100644
index 0000000000000..4cdf7737e251f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf5-3-d23017942dc49be9f5a61430201371bf
@@ -0,0 +1 @@
+2008-11-11 15:32:20	2008-11-11	1	11	2008	1	11	2008
diff --git a/sql/hive/src/test/resources/golden/udf5-4-1b35f4ee3febf99804db1f481af80b23 b/sql/hive/src/test/resources/golden/udf5-4-1b35f4ee3febf99804db1f481af80b23
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/udf5-5-2125da6f09799cf7f10b838fc8f24e71 b/sql/hive/src/test/resources/golden/udf5-5-2125da6f09799cf7f10b838fc8f24e71
new file mode 100644
index 0000000000000..bfd616764b235
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf5-5-2125da6f09799cf7f10b838fc8f24e71
@@ -0,0 +1 @@
+01/13/10 11:57:40	2010-01-13 11:57:40
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_java_method-0-50131c0ba7b7a6b65c789a5a8497bada
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_java_method-0-50131c0ba7b7a6b65c789a5a8497bada
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 b/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7
new file mode 100644
index 0000000000000..91e538becfc96
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7
@@ -0,0 +1 @@
+java_method(class,method[,arg1[,arg2..]]) calls method with reflection
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-1-991b98a25032b21802bc2a1efde606c7 b/sql/hive/src/test/resources/golden/udf_java_method-1-991b98a25032b21802bc2a1efde606c7
new file mode 100644
index 0000000000000..91e538becfc96
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_java_method-1-991b98a25032b21802bc2a1efde606c7
@@ -0,0 +1 @@
+java_method(class,method[,arg1[,arg2..]]) calls method with reflection
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a b/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a
new file mode 100644
index 0000000000000..6315f678b46f8
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a
@@ -0,0 +1,4 @@
+java_method(class,method[,arg1[,arg2..]]) calls method with reflection
+Synonyms: reflect
+Use this UDF to call Java methods by matching the argument signature
+
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 b/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-2-a3b94d9f2c2caf85a588b6686a64630a b/sql/hive/src/test/resources/golden/udf_java_method-2-a3b94d9f2c2caf85a588b6686a64630a
new file mode 100644
index 0000000000000..6315f678b46f8
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_java_method-2-a3b94d9f2c2caf85a588b6686a64630a
@@ -0,0 +1,4 @@
+java_method(class,method[,arg1[,arg2..]]) calls method with reflection
+Synonyms: reflect
+Use this UDF to call Java methods by matching the argument signature
+
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-3-9ae6daaf9783d3d6577231320727582a b/sql/hive/src/test/resources/golden/udf_java_method-3-9ae6daaf9783d3d6577231320727582a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd b/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd
new file mode 100644
index 0000000000000..51ff65ea1870f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd
@@ -0,0 +1 @@
+1	true	3	2	3	2.718281828459045	1.0
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-4-2fc4554258492a1d92c89a8dbad6c1c3 b/sql/hive/src/test/resources/golden/udf_java_method-4-2fc4554258492a1d92c89a8dbad6c1c3
new file mode 100644
index 0000000000000..51ff65ea1870f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_java_method-4-2fc4554258492a1d92c89a8dbad6c1c3
@@ -0,0 +1 @@
+1	true	3	2	3	2.718281828459045	1.0
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_pmod-0-50131c0ba7b7a6b65c789a5a8497bada
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-0-50131c0ba7b7a6b65c789a5a8497bada
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 b/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1
new file mode 100644
index 0000000000000..5d2fc352ee060
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1
@@ -0,0 +1 @@
+a pmod b - Compute the positive modulo
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae b/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae
new file mode 100644
index 0000000000000..5d2fc352ee060
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae
@@ -0,0 +1 @@
+a pmod b - Compute the positive modulo
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-1-ed67184beaf84c0542117c26651938e1 b/sql/hive/src/test/resources/golden/udf_pmod-1-ed67184beaf84c0542117c26651938e1
new file mode 100644
index 0000000000000..5d2fc352ee060
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-1-ed67184beaf84c0542117c26651938e1
@@ -0,0 +1 @@
+a pmod b - Compute the positive modulo
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 b/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2
new file mode 100644
index 0000000000000..0b46af11c4516
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2
@@ -0,0 +1 @@
+6.89	51.7	18.09
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-10-cab94a0c6b36a489aab9f3f305b92772 b/sql/hive/src/test/resources/golden/udf_pmod-10-cab94a0c6b36a489aab9f3f305b92772
new file mode 100644
index 0000000000000..ab842acd48b3c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-10-cab94a0c6b36a489aab9f3f305b92772
@@ -0,0 +1 @@
+6.890000000000011	51.699999999999996	18.090000000000003
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-11-7ca6baa647215c334419d1bb8a527449 b/sql/hive/src/test/resources/golden/udf_pmod-11-7ca6baa647215c334419d1bb8a527449
new file mode 100644
index 0000000000000..0b46af11c4516
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-11-7ca6baa647215c334419d1bb8a527449
@@ -0,0 +1 @@
+6.89	51.7	18.09
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 b/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-2-90f75e01dcee85253a501d53b8562dae b/sql/hive/src/test/resources/golden/udf_pmod-2-90f75e01dcee85253a501d53b8562dae
new file mode 100644
index 0000000000000..5d2fc352ee060
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-2-90f75e01dcee85253a501d53b8562dae
@@ -0,0 +1 @@
+a pmod b - Compute the positive modulo
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 b/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234
new file mode 100644
index 0000000000000..5eb0813b60eb6
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234
@@ -0,0 +1 @@
+8	51	15
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-3-8d6dd8a5e7a519fdc5261e4193e3464f b/sql/hive/src/test/resources/golden/udf_pmod-3-8d6dd8a5e7a519fdc5261e4193e3464f
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-3-8d6dd8a5e7a519fdc5261e4193e3464f
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-4-608e04ca8855780fb9e60486759b19b8 b/sql/hive/src/test/resources/golden/udf_pmod-4-608e04ca8855780fb9e60486759b19b8
new file mode 100644
index 0000000000000..5eb0813b60eb6
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-4-608e04ca8855780fb9e60486759b19b8
@@ -0,0 +1 @@
+8	51	15
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 b/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1
new file mode 100644
index 0000000000000..e21e4b08e7a62
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1
@@ -0,0 +1 @@
+5	50	0
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 b/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029
new file mode 100644
index 0000000000000..e0bc2a844fb46
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029
@@ -0,0 +1 @@
+8	51	16
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-5-e3ca9fe032dd6f71e33ddf367ef5e2cf b/sql/hive/src/test/resources/golden/udf_pmod-5-e3ca9fe032dd6f71e33ddf367ef5e2cf
new file mode 100644
index 0000000000000..e21e4b08e7a62
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-5-e3ca9fe032dd6f71e33ddf367ef5e2cf
@@ -0,0 +1 @@
+5	50	0
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 b/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125
new file mode 100644
index 0000000000000..e0bc2a844fb46
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125
@@ -0,0 +1 @@
+8	51	16
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-6-c5c810e71bed8e56c1bac59b7d9c16c5 b/sql/hive/src/test/resources/golden/udf_pmod-6-c5c810e71bed8e56c1bac59b7d9c16c5
new file mode 100644
index 0000000000000..e0bc2a844fb46
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-6-c5c810e71bed8e56c1bac59b7d9c16c5
@@ -0,0 +1 @@
+8	51	16
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-7-3665a6414590bb2aff522dfe847dbc0e b/sql/hive/src/test/resources/golden/udf_pmod-7-3665a6414590bb2aff522dfe847dbc0e
new file mode 100644
index 0000000000000..e0bc2a844fb46
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-7-3665a6414590bb2aff522dfe847dbc0e
@@ -0,0 +1 @@
+8	51	16
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 b/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618
new file mode 100644
index 0000000000000..e0bc2a844fb46
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618
@@ -0,0 +1 @@
+8	51	16
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-8-95f85c34076952af0640b596365b27ca b/sql/hive/src/test/resources/golden/udf_pmod-8-95f85c34076952af0640b596365b27ca
new file mode 100644
index 0000000000000..e0bc2a844fb46
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-8-95f85c34076952af0640b596365b27ca
@@ -0,0 +1 @@
+8	51	16
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd b/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd
new file mode 100644
index 0000000000000..48371142e9b5d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd
@@ -0,0 +1 @@
+6.8899984	51.700005	18.089996
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-9-798ef5064b61d0ae403e3e11c8fd749b b/sql/hive/src/test/resources/golden/udf_pmod-9-798ef5064b61d0ae403e3e11c8fd749b
new file mode 100644
index 0000000000000..48371142e9b5d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-9-798ef5064b61d0ae403e3e11c8fd749b
@@ -0,0 +1 @@
+6.8899984	51.700005	18.089996
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 b/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57
new file mode 100644
index 0000000000000..ab842acd48b3c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57
@@ -0,0 +1 @@
+6.890000000000011	51.699999999999996	18.090000000000003
diff --git a/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 b/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15
new file mode 100644
index 0000000000000..49fdc0a774e70
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15
@@ -0,0 +1 @@
+round(x[, d]) - round x to d decimal places
diff --git a/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 b/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8
new file mode 100644
index 0000000000000..862adeae821ff
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8
@@ -0,0 +1,4 @@
+round(x[, d]) - round x to d decimal places
+Example:
+  > SELECT round(12.3456, 1) FROM src LIMIT 1;
+  12.3'
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_to_double-0-50131c0ba7b7a6b65c789a5a8497bada
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-0-50131c0ba7b7a6b65c789a5a8497bada
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 b/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a b/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a
new file mode 100644
index 0000000000000..d3827e75a5cad
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a
@@ -0,0 +1 @@
+1.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-1-9d7f149dc28bd312425392c3f2abea72 b/sql/hive/src/test/resources/golden/udf_to_double-1-9d7f149dc28bd312425392c3f2abea72
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-1-9d7f149dc28bd312425392c3f2abea72
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe b/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe
new file mode 100644
index 0000000000000..319fde05380bc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe
@@ -0,0 +1 @@
+-7.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-2-e5df309104b260ff9145229d119a774d b/sql/hive/src/test/resources/golden/udf_to_double-2-e5df309104b260ff9145229d119a774d
new file mode 100644
index 0000000000000..d3827e75a5cad
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-2-e5df309104b260ff9145229d119a774d
@@ -0,0 +1 @@
+1.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f b/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f
new file mode 100644
index 0000000000000..8c1c4fe62b6c2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f
@@ -0,0 +1 @@
+-18.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-3-abd1a8fc84fcd692891c1ac242492e45 b/sql/hive/src/test/resources/golden/udf_to_double-3-abd1a8fc84fcd692891c1ac242492e45
new file mode 100644
index 0000000000000..319fde05380bc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-3-abd1a8fc84fcd692891c1ac242492e45
@@ -0,0 +1 @@
+-7.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 b/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68
new file mode 100644
index 0000000000000..1b650de78904f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68
@@ -0,0 +1 @@
+-129.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-4-3651962b1a5fac4f1dc02f0403e68471 b/sql/hive/src/test/resources/golden/udf_to_double-4-3651962b1a5fac4f1dc02f0403e68471
new file mode 100644
index 0000000000000..8c1c4fe62b6c2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-4-3651962b1a5fac4f1dc02f0403e68471
@@ -0,0 +1 @@
+-18.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-5-39bd92a64ad9d5f57d477bf668e08da5 b/sql/hive/src/test/resources/golden/udf_to_double-5-39bd92a64ad9d5f57d477bf668e08da5
new file mode 100644
index 0000000000000..1b650de78904f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-5-39bd92a64ad9d5f57d477bf668e08da5
@@ -0,0 +1 @@
+-129.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 b/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8
new file mode 100644
index 0000000000000..3a3bd0df03b5b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8
@@ -0,0 +1 @@
+-1025.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 b/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6
new file mode 100644
index 0000000000000..38f7ad5afa0ab
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6
@@ -0,0 +1 @@
+-3.140000104904175
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-6-b65b3f3b72ce068c2b954850fe5fc2a6 b/sql/hive/src/test/resources/golden/udf_to_double-6-b65b3f3b72ce068c2b954850fe5fc2a6
new file mode 100644
index 0000000000000..3a3bd0df03b5b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-6-b65b3f3b72ce068c2b954850fe5fc2a6
@@ -0,0 +1 @@
+-1025.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 b/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5
new file mode 100644
index 0000000000000..01e913dbfe725
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5
@@ -0,0 +1 @@
+-3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-7-ab78e74674e92847fd44db0d21c2a145 b/sql/hive/src/test/resources/golden/udf_to_double-7-ab78e74674e92847fd44db0d21c2a145
new file mode 100644
index 0000000000000..38f7ad5afa0ab
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-7-ab78e74674e92847fd44db0d21c2a145
@@ -0,0 +1 @@
+-3.140000104904175
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-8-22e97175b71ca7fd8668130f5a757aee b/sql/hive/src/test/resources/golden/udf_to_double-8-22e97175b71ca7fd8668130f5a757aee
new file mode 100644
index 0000000000000..01e913dbfe725
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-8-22e97175b71ca7fd8668130f5a757aee
@@ -0,0 +1 @@
+-3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 b/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982
new file mode 100644
index 0000000000000..f45d1f04dc920
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982
@@ -0,0 +1 @@
+-38.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-9-53d0629f93ae811965bb4658e1aa3cb9 b/sql/hive/src/test/resources/golden/udf_to_double-9-53d0629f93ae811965bb4658e1aa3cb9
new file mode 100644
index 0000000000000..f45d1f04dc920
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_double-9-53d0629f93ae811965bb4658e1aa3cb9
@@ -0,0 +1 @@
+-38.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_to_float-0-50131c0ba7b7a6b65c789a5a8497bada
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-0-50131c0ba7b7a6b65c789a5a8497bada
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 b/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-1-6f0f1660c78aa1318ae1da4a2afdd9dd b/sql/hive/src/test/resources/golden/udf_to_float-1-6f0f1660c78aa1318ae1da4a2afdd9dd
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-1-6f0f1660c78aa1318ae1da4a2afdd9dd
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c b/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c
new file mode 100644
index 0000000000000..d3827e75a5cad
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c
@@ -0,0 +1 @@
+1.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 b/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971
new file mode 100644
index 0000000000000..319fde05380bc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971
@@ -0,0 +1 @@
+-7.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-2-ef6719eced842e7efe970665b41f8c0 b/sql/hive/src/test/resources/golden/udf_to_float-2-ef6719eced842e7efe970665b41f8c0
new file mode 100644
index 0000000000000..d3827e75a5cad
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-2-ef6719eced842e7efe970665b41f8c0
@@ -0,0 +1 @@
+1.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce b/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce
new file mode 100644
index 0000000000000..8c1c4fe62b6c2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce
@@ -0,0 +1 @@
+-18.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-3-bfb661d2179679c317a7b088837258d3 b/sql/hive/src/test/resources/golden/udf_to_float-3-bfb661d2179679c317a7b088837258d3
new file mode 100644
index 0000000000000..319fde05380bc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-3-bfb661d2179679c317a7b088837258d3
@@ -0,0 +1 @@
+-7.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-4-477519ea558ef60feb754d442c1b13cc b/sql/hive/src/test/resources/golden/udf_to_float-4-477519ea558ef60feb754d442c1b13cc
new file mode 100644
index 0000000000000..8c1c4fe62b6c2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-4-477519ea558ef60feb754d442c1b13cc
@@ -0,0 +1 @@
+-18.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 b/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507
new file mode 100644
index 0000000000000..1b650de78904f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507
@@ -0,0 +1 @@
+-129.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-5-3ea0b65c600c2a6c0a2f20b36bc02c0a b/sql/hive/src/test/resources/golden/udf_to_float-5-3ea0b65c600c2a6c0a2f20b36bc02c0a
new file mode 100644
index 0000000000000..1b650de78904f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-5-3ea0b65c600c2a6c0a2f20b36bc02c0a
@@ -0,0 +1 @@
+-129.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 b/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201
new file mode 100644
index 0000000000000..3a3bd0df03b5b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201
@@ -0,0 +1 @@
+-1025.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 b/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002
new file mode 100644
index 0000000000000..01e913dbfe725
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002
@@ -0,0 +1 @@
+-3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-6-6bf596b8ac0a57d7df844cca1c94a0c7 b/sql/hive/src/test/resources/golden/udf_to_float-6-6bf596b8ac0a57d7df844cca1c94a0c7
new file mode 100644
index 0000000000000..3a3bd0df03b5b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-6-6bf596b8ac0a57d7df844cca1c94a0c7
@@ -0,0 +1 @@
+-1025.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 b/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1
new file mode 100644
index 0000000000000..01e913dbfe725
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1
@@ -0,0 +1 @@
+-3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-7-e4b449ba415538aac9c9ec421d8bcce8 b/sql/hive/src/test/resources/golden/udf_to_float-7-e4b449ba415538aac9c9ec421d8bcce8
new file mode 100644
index 0000000000000..01e913dbfe725
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-7-e4b449ba415538aac9c9ec421d8bcce8
@@ -0,0 +1 @@
+-3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c b/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c
new file mode 100644
index 0000000000000..f45d1f04dc920
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c
@@ -0,0 +1 @@
+-38.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-8-c46d1c40e52bef886e56ae1e07892bb7 b/sql/hive/src/test/resources/golden/udf_to_float-8-c46d1c40e52bef886e56ae1e07892bb7
new file mode 100644
index 0000000000000..01e913dbfe725
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-8-c46d1c40e52bef886e56ae1e07892bb7
@@ -0,0 +1 @@
+-3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-9-c5545924be7d13b1f4a13cb2bd0c17cc b/sql/hive/src/test/resources/golden/udf_to_float-9-c5545924be7d13b1f4a13cb2bd0c17cc
new file mode 100644
index 0000000000000..f45d1f04dc920
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_float-9-c5545924be7d13b1f4a13cb2bd0c17cc
@@ -0,0 +1 @@
+-38.14
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
new file mode 100644
index 0000000000000..f90d3607915ae
--- /dev/null
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
@@ -0,0 +1,226 @@
+/*
+ * 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.sql.hive
+
+import java.sql.Date
+import java.util
+import java.util.{Locale, TimeZone}
+
+import org.apache.hadoop.hive.serde2.io.DoubleWritable
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
+import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.scalatest.FunSuite
+
+import org.apache.hadoop.hive.ql.udf.UDAFPercentile
+import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector, ObjectInspectorFactory}
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions
+import org.apache.hadoop.io.LongWritable
+
+import org.apache.spark.sql.catalyst.expressions.{Literal, Row}
+
+class HiveInspectorSuite extends FunSuite with HiveInspectors {
+  test("Test wrap SettableStructObjectInspector") {
+    val udaf = new UDAFPercentile.PercentileLongEvaluator()
+    udaf.init()
+
+    udaf.iterate(new LongWritable(1), 0.1)
+    udaf.iterate(new LongWritable(1), 0.1)
+
+    val state = udaf.terminatePartial()
+
+    val soi = ObjectInspectorFactory.getReflectionObjectInspector(
+      classOf[UDAFPercentile.State],
+      ObjectInspectorOptions.JAVA).asInstanceOf[StructObjectInspector]
+
+    val a = unwrap(state, soi).asInstanceOf[Row]
+    val b = wrap(a, soi).asInstanceOf[UDAFPercentile.State]
+
+    val sfCounts = soi.getStructFieldRef("counts")
+    val sfPercentiles = soi.getStructFieldRef("percentiles")
+
+    assert(2 === soi.getStructFieldData(b, sfCounts)
+      .asInstanceOf[util.Map[LongWritable, LongWritable]]
+      .get(new LongWritable(1L))
+      .get())
+    assert(0.1 === soi.getStructFieldData(b, sfPercentiles)
+      .asInstanceOf[util.ArrayList[DoubleWritable]]
+      .get(0)
+      .get())
+  }
+
+  // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
+  TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
+  // Add Locale setting
+  Locale.setDefault(Locale.US)
+
+  val data =
+    Literal(true) ::
+    Literal(0.asInstanceOf[Byte]) ::
+    Literal(0.asInstanceOf[Short]) ::
+    Literal(0) ::
+    Literal(0.asInstanceOf[Long]) ::
+    Literal(0.asInstanceOf[Float]) ::
+    Literal(0.asInstanceOf[Double]) ::
+    Literal("0") ::
+    Literal(new Date(2014, 9, 23)) ::
+    Literal(Decimal(BigDecimal(123.123))) ::
+    Literal(new java.sql.Timestamp(123123)) ::
+    Literal(Array[Byte](1,2,3)) ::
+    Literal(Seq[Int](1,2,3), ArrayType(IntegerType)) ::
+    Literal(Map[Int, Int](1->2, 2->1), MapType(IntegerType, IntegerType)) ::
+    Literal(Row(1,2.0d,3.0f),
+      StructType(StructField("c1", IntegerType) ::
+      StructField("c2", DoubleType) ::
+      StructField("c3", FloatType) :: Nil)) ::
+    Nil
+
+  val row = data.map(_.eval(null))
+  val dataTypes = data.map(_.dataType)
+
+  import scala.collection.JavaConversions._
+  def toWritableInspector(dataType: DataType): ObjectInspector = dataType match {
+    case ArrayType(tpe, _) =>
+      ObjectInspectorFactory.getStandardListObjectInspector(toWritableInspector(tpe))
+    case MapType(keyType, valueType, _) =>
+      ObjectInspectorFactory.getStandardMapObjectInspector(
+        toWritableInspector(keyType), toWritableInspector(valueType))
+    case StringType => PrimitiveObjectInspectorFactory.writableStringObjectInspector
+    case IntegerType => PrimitiveObjectInspectorFactory.writableIntObjectInspector
+    case DoubleType => PrimitiveObjectInspectorFactory.writableDoubleObjectInspector
+    case BooleanType => PrimitiveObjectInspectorFactory.writableBooleanObjectInspector
+    case LongType => PrimitiveObjectInspectorFactory.writableLongObjectInspector
+    case FloatType => PrimitiveObjectInspectorFactory.writableFloatObjectInspector
+    case ShortType => PrimitiveObjectInspectorFactory.writableShortObjectInspector
+    case ByteType => PrimitiveObjectInspectorFactory.writableByteObjectInspector
+    case NullType => PrimitiveObjectInspectorFactory.writableVoidObjectInspector
+    case BinaryType => PrimitiveObjectInspectorFactory.writableBinaryObjectInspector
+    case DateType => PrimitiveObjectInspectorFactory.writableDateObjectInspector
+    case TimestampType => PrimitiveObjectInspectorFactory.writableTimestampObjectInspector
+    case DecimalType() => PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector
+    case StructType(fields) =>
+      ObjectInspectorFactory.getStandardStructObjectInspector(
+        fields.map(f => f.name), fields.map(f => toWritableInspector(f.dataType)))
+  }
+
+  def checkDataType(dt1: Seq[DataType], dt2: Seq[DataType]): Unit = {
+    dt1.zip(dt2).map {
+      case (dd1, dd2) =>
+        assert(dd1.getClass === dd2.getClass)  // DecimalType doesn't has the default precision info
+    }
+  }
+
+  def checkValues(row1: Seq[Any], row2: Seq[Any]): Unit = {
+    row1.zip(row2).map {
+      case (r1, r2) => checkValue(r1, r2)
+    }
+  }
+
+  def checkValue(v1: Any, v2: Any): Unit = {
+    (v1, v2) match {
+      case (r1: Decimal, r2: Decimal) =>
+        // Ignore the Decimal precision
+        assert(r1.compare(r2) === 0)
+      case (r1: Array[Byte], r2: Array[Byte])
+        if r1 != null && r2 != null && r1.length == r2.length =>
+        r1.zip(r2).map { case (b1, b2) => assert(b1 === b2) }
+      case (r1: Date, r2: Date) => assert(r1.compareTo(r2) === 0)
+      case (r1, r2) => assert(r1 === r2)
+    }
+  }
+
+  test("oi => datatype => oi") {
+    val ois = dataTypes.map(toInspector)
+
+    checkDataType(ois.map(inspectorToDataType), dataTypes)
+    checkDataType(dataTypes.map(toWritableInspector).map(inspectorToDataType), dataTypes)
+  }
+
+  test("wrap / unwrap null, constant null and writables") {
+    val writableOIs = dataTypes.map(toWritableInspector)
+    val nullRow = data.map(d => null)
+
+    checkValues(nullRow, nullRow.zip(writableOIs).map {
+      case (d, oi) => unwrap(wrap(d, oi), oi)
+    })
+
+    // struct couldn't be constant, sweep it out
+    val constantExprs = data.filter(!_.dataType.isInstanceOf[StructType])
+    val constantData = constantExprs.map(_.eval())
+    val constantNullData = constantData.map(_ => null)
+    val constantWritableOIs = constantExprs.map(e => toWritableInspector(e.dataType))
+    val constantNullWritableOIs = constantExprs.map(e => toInspector(Literal(null, e.dataType)))
+
+    checkValues(constantData, constantData.zip(constantWritableOIs).map {
+      case (d, oi) => unwrap(wrap(d, oi), oi)
+    })
+
+    checkValues(constantNullData, constantData.zip(constantNullWritableOIs).map {
+      case (d, oi) => unwrap(wrap(d, oi), oi)
+    })
+
+    checkValues(constantNullData, constantNullData.zip(constantWritableOIs).map {
+      case (d, oi) => unwrap(wrap(d, oi), oi)
+    })
+  }
+
+  test("wrap / unwrap primitive writable object inspector") {
+    val writableOIs = dataTypes.map(toWritableInspector)
+
+    checkValues(row, row.zip(writableOIs).map {
+      case (data, oi) => unwrap(wrap(data, oi), oi)
+    })
+  }
+
+  test("wrap / unwrap primitive java object inspector") {
+    val ois = dataTypes.map(toInspector)
+
+    checkValues(row, row.zip(ois).map {
+      case (data, oi) => unwrap(wrap(data, oi), oi)
+    })
+  }
+
+  test("wrap / unwrap Struct Type") {
+    val dt = StructType(dataTypes.zipWithIndex.map {
+      case (t, idx) => StructField(s"c_$idx", t)
+    })
+
+    checkValues(row, unwrap(wrap(row, toInspector(dt)), toInspector(dt)).asInstanceOf[Row])
+    checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt)))
+  }
+
+  test("wrap / unwrap Array Type") {
+    val dt = ArrayType(dataTypes(0))
+
+    val d = row(0) :: row(0) :: Nil
+    checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt)))
+    checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt)))
+    checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt))))
+    checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt))))
+  }
+
+  test("wrap / unwrap Map Type") {
+    val dt = MapType(dataTypes(0), dataTypes(1))
+
+    val d = Map(row(0) -> row(1))
+    checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt)))
+    checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt)))
+    checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt))))
+    checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt))))
+  }
+}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
index a90fc023e67d8..4b6a9308b9811 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
@@ -22,10 +22,10 @@ import org.scalatest.BeforeAndAfterAll
 import scala.reflect.ClassTag
 
 import org.apache.spark.sql.{SQLConf, QueryTest}
-import org.apache.spark.sql.catalyst.plans.logical.NativeCommand
-import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin}
+import org.apache.spark.sql.execution.joins._
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
+import org.apache.spark.sql.hive.execution._
 
 class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
   TestHive.reset()
@@ -51,19 +51,19 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
 
     assertAnalyzeCommand(
       "ANALYZE TABLE Table1 COMPUTE STATISTICS",
-      classOf[NativeCommand])
+      classOf[HiveNativeCommand])
     assertAnalyzeCommand(
       "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS",
-      classOf[NativeCommand])
+      classOf[HiveNativeCommand])
     assertAnalyzeCommand(
       "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan",
-      classOf[NativeCommand])
+      classOf[HiveNativeCommand])
     assertAnalyzeCommand(
       "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS",
-      classOf[NativeCommand])
+      classOf[HiveNativeCommand])
     assertAnalyzeCommand(
       "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS noscan",
-      classOf[NativeCommand])
+      classOf[HiveNativeCommand])
 
     assertAnalyzeCommand(
       "ANALYZE TABLE Table1 COMPUTE STATISTICS nOscAn",
@@ -193,4 +193,52 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
     )
   }
 
+  test("auto converts to broadcast left semi join, by size estimate of a relation") {
+    val leftSemiJoinQuery =
+      """SELECT * FROM src a
+        |left semi JOIN src b ON a.key=86 and a.key = b.key""".stripMargin
+    val answer = (86, "val_86") :: Nil
+
+    var rdd = sql(leftSemiJoinQuery)
+
+    // Assert src has a size smaller than the threshold.
+    val sizes = rdd.queryExecution.analyzed.collect {
+      case r if implicitly[ClassTag[MetastoreRelation]].runtimeClass
+        .isAssignableFrom(r.getClass) =>
+        r.statistics.sizeInBytes
+    }
+    assert(sizes.size === 2 && sizes(1) <= autoBroadcastJoinThreshold
+      && sizes(0) <= autoBroadcastJoinThreshold,
+      s"query should contain two relations, each of which has size smaller than autoConvertSize")
+
+    // Using `sparkPlan` because for relevant patterns in HashJoin to be
+    // matched, other strategies need to be applied.
+    var bhj = rdd.queryExecution.sparkPlan.collect {
+      case j: BroadcastLeftSemiJoinHash => j
+    }
+    assert(bhj.size === 1,
+      s"actual query plans do not contain broadcast join: ${rdd.queryExecution}")
+
+    checkAnswer(rdd, answer) // check correctness of output
+
+    TestHive.settings.synchronized {
+      val tmp = autoBroadcastJoinThreshold
+
+      sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1")
+      rdd = sql(leftSemiJoinQuery)
+      bhj = rdd.queryExecution.sparkPlan.collect {
+        case j: BroadcastLeftSemiJoinHash => j
+      }
+      assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off")
+
+      val shj = rdd.queryExecution.sparkPlan.collect {
+        case j: LeftSemiJoinHash => j
+      }
+      assert(shj.size === 1,
+        "LeftSemiJoinHash should be planned when BroadcastHashJoin is turned off")
+
+      sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp")
+    }
+
+  }
 }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index 44eb4cfa59335..4104df8f8e022 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -24,7 +24,6 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen}
 import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.planning.PhysicalOperation
 import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand => LogicalNativeCommand}
 import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.hive.test.TestHive
 
@@ -133,7 +132,7 @@ abstract class HiveComparisonTest
 
     def isSorted(plan: LogicalPlan): Boolean = plan match {
       case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false
-      case PhysicalOperation(_, _, Sort(_, _)) => true
+      case PhysicalOperation(_, _, Sort(_, true, _)) => true
       case _ => plan.children.iterator.exists(isSorted)
     }
 
@@ -142,14 +141,14 @@ abstract class HiveComparisonTest
       // Hack: Hive simply prints the result of a SET command to screen,
       // and does not return it as a query answer.
       case _: SetCommand => Seq("0")
-      case LogicalNativeCommand(c) if c.toLowerCase.contains("desc") =>
+      case HiveNativeCommand(c) if c.toLowerCase.contains("desc") =>
         answer
           .filterNot(nonDeterministicLine)
           .map(_.replaceAll("from deserializer", ""))
           .map(_.replaceAll("None", ""))
           .map(_.trim)
           .filterNot(_ == "")
-      case _: LogicalNativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "")
+      case _: HiveNativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "")
       case _: ExplainCommand => answer
       case _: DescribeCommand =>
         // Filter out non-deterministic lines and lines which do not have actual results but
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index af45dfd6e28c2..fb6da33e88ef6 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -56,6 +56,13 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
     Locale.setDefault(originalLocale)
   }
 
+  test("SPARK-4908: concurent hive native commands") {
+    (1 to 100).par.map { _ =>
+      sql("USE default")
+      sql("SHOW TABLES")
+    }
+  }
+
   createQueryTest("constant object inspector for generic udf",
     """SELECT named_struct(
       lower("AA"), "10",
@@ -319,10 +326,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
   createQueryTest("DISTINCT",
     "SELECT DISTINCT key, value FROM src")
 
-  ignore("empty aggregate input") {
-    createQueryTest("empty aggregate input",
-      "SELECT SUM(key) FROM (SELECT * FROM src LIMIT 0) a")
-  }
+  createQueryTest("empty aggregate input",
+    "SELECT SUM(key) FROM (SELECT * FROM src LIMIT 0) a")
 
   createQueryTest("lateral view1",
     "SELECT tbl.* FROM src LATERAL VIEW explode(array(1,2)) tbl as a")
@@ -413,6 +418,15 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
   createQueryTest("select null from table",
     "SELECT null FROM src LIMIT 1")
 
+  test("predicates contains an empty AttributeSet() references") {
+    sql(
+      """
+        |SELECT a FROM (
+        |  SELECT 1 AS a FROM src LIMIT 1 ) table
+        |WHERE abs(20141202) is not null
+      """.stripMargin).collect()
+  }
+
   test("implement identity function using case statement") {
     val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src")
       .map { case Row(i: Int) => i }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index ee9d08ff75450..422e843d2b0d2 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -27,6 +27,17 @@ case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested])
  * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution.
  */
 class HiveResolutionSuite extends HiveComparisonTest {
+
+  case class NestedData(a: Seq[NestedData2], B: NestedData2)
+  case class NestedData2(a: NestedData3, B: NestedData3)
+  case class NestedData3(a: Int, B: Int)
+
+  test("SPARK-3698: case insensitive test for nested data") {
+    sparkContext.makeRDD(Seq.empty[NestedData]).registerTempTable("nested")
+    // This should be successfully analyzed
+    sql("SELECT a[0].A.A from nested").queryExecution.analyzed
+  }
+
   createQueryTest("table.attr",
     "SELECT src.key FROM src ORDER BY key LIMIT 1")
 
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
index 54c0f017d4cb6..16f77a438e1ae 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
@@ -18,7 +18,8 @@
 package org.apache.spark.sql.hive.execution
 
 import org.apache.spark.sql.hive.test.TestHive
-import org.apache.spark.sql.{Row, SchemaRDD}
+import org.apache.spark.sql.hive.test.TestHive._
+import org.apache.spark.sql.Row
 
 import org.apache.spark.util.Utils
 
@@ -44,6 +45,14 @@ class HiveTableScanSuite extends HiveComparisonTest {
       |SELECT * from part_scan_test;
     """.stripMargin)
 
+  // In unit test, kv1.txt is a small file and will be loaded as table src
+  // Since the small file will be considered as a single split, we assume
+  // Hive / SparkSQL HQL has the same output even for SORT BY
+  createQueryTest("file_split_for_small_table",
+    """
+      |SELECT key, value FROM src SORT BY key, value
+    """.stripMargin)
+
   test("Spark-4041: lowercase issue") {
     TestHive.sql("CREATE TABLE tb (KEY INT, VALUE STRING) STORED AS ORC")
     TestHive.sql("insert into table tb select key, value from src")
@@ -68,5 +77,15 @@ class HiveTableScanSuite extends HiveComparisonTest {
       === Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")),Row(null)))
     TestHive.sql("DROP TABLE timestamp_query_null")
   }
-  
+
+  test("Spark-4959 Attributes are case sensitive when using a select query from a projection") {
+    sql("create table spark_4959 (col1 string)")
+    sql("""insert into table spark_4959 select "hi" from src limit 1""")
+    table("spark_4959").select(
+      'col1.as('CaseSensitiveColName),
+      'col1.as('CaseSensitiveColName2)).registerTempTable("spark_4959_2")
+
+    assert(sql("select CaseSensitiveColName from spark_4959_2").first() === Row("hi"))
+    assert(sql("select casesensitivecolname from spark_4959_2").first() === Row("hi"))
+  }
 }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
index 5fcaf671a80de..5fc8d8dbe3a9f 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
@@ -92,10 +92,21 @@ class HiveUdfSuite extends QueryTest {
   }
 
   test("SPARK-2693 udaf aggregates test") {
-    checkAnswer(sql("SELECT percentile(key,1) FROM src LIMIT 1"),
+    checkAnswer(sql("SELECT percentile(key, 1) FROM src LIMIT 1"),
       sql("SELECT max(key) FROM src").collect().toSeq)
+      
+    checkAnswer(sql("SELECT percentile(key, array(1, 1)) FROM src LIMIT 1"),
+      sql("SELECT array(max(key), max(key)) FROM src").collect().toSeq)
   }
 
+  test("Generic UDAF aggregates") {
+    checkAnswer(sql("SELECT ceiling(percentile_approx(key, 0.99999)) FROM src LIMIT 1"),
+      sql("SELECT max(key) FROM src LIMIT 1").collect().toSeq)
+      
+    checkAnswer(sql("SELECT percentile_approx(100.0, array(0.9, 0.9)) FROM src LIMIT 1"),
+      sql("SELECT array(100, 100) FROM src LIMIT 1").collect().toSeq)
+   }
+  
   test("UDFIntegerToString") {
     val testData = TestHive.sparkContext.parallelize(
       IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index b341eae512417..5d0fb7237011f 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -32,6 +32,13 @@ case class Nested3(f3: Int)
  * valid, but Hive currently cannot execute it.
  */
 class SQLQuerySuite extends QueryTest {
+  test("SPARK-4512 Fix attribute reference resolution error when using SORT BY") {
+    checkAnswer(
+      sql("SELECT * FROM (SELECT key + key AS a FROM src SORT BY value) t ORDER BY t.a"),
+      sql("SELECT key + key as a FROM src ORDER BY a").collect().toSeq
+    )
+  }
+
   test("CTAS with serde") {
     sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect
     sql(
@@ -137,6 +144,19 @@ class SQLQuerySuite extends QueryTest {
       sql("SELECT key, value FROM src ORDER BY key").collect().toSeq)
   }
 
+  test("SPARK-4825 save join to table") {
+    val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString))
+    sql("CREATE TABLE test1 (key INT, value STRING)")
+    testData.insertInto("test1")
+    sql("CREATE TABLE test2 (key INT, value STRING)")
+    testData.insertInto("test2")
+    testData.insertInto("test2")
+    sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").saveAsTable("test")
+    checkAnswer(
+      table("test"),
+      sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq)
+  }
+
   test("SPARK-3708 Backticks aren't handled correctly is aliases") {
     checkAnswer(
       sql("SELECT k FROM (SELECT `key` AS `k` FROM src) a"),
@@ -172,9 +192,14 @@ class SQLQuerySuite extends QueryTest {
       sql("SELECT case when ~1=-2 then 1 else 0 end FROM src"),
       sql("SELECT 1 FROM src").collect().toSeq)
   }
-  
- test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") {
-    checkAnswer(sql("SELECT key FROM src WHERE key not between 0 and 10 order by key"), 
-        sql("SELECT key FROM src WHERE key between 11 and 500 order by key").collect().toSeq)
+
+  test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") {
+    checkAnswer(sql("SELECT key FROM src WHERE key not between 0 and 10 order by key"),
+      sql("SELECT key FROM src WHERE key between 11 and 500 order by key").collect().toSeq)
+  }
+
+  test("SPARK-2554 SumDistinct partial aggregation") {
+    checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"),
+      sql("SELECT distinct key FROM src order by key").collect().toSeq)
   }
 }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
index 6f57fe8958387..4bc14bad0ad5f 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
@@ -17,103 +17,66 @@
 
 package org.apache.spark.sql.parquet
 
-import java.io.File
-
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-
-import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row}
-import org.apache.spark.sql.catalyst.types.{DataType, StringType, IntegerType}
-import org.apache.spark.sql.{parquet, SchemaRDD}
-import org.apache.spark.util.Utils
-
-// Implicits
-import org.apache.spark.sql.hive.test.TestHive._
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.hive.test.TestHive
 
 case class Cases(lower: String, UPPER: String)
 
-class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach {
-
-  val dirname = Utils.createTempDir()
-
-  var testRDD: SchemaRDD = null
-
-  override def beforeAll() {
-    // write test data
-    ParquetTestData.writeFile()
-    testRDD = parquetFile(ParquetTestData.testDir.toString)
-    testRDD.registerTempTable("testsource")
-  }
-
-  override def afterAll() {
-    Utils.deleteRecursively(ParquetTestData.testDir)
-    Utils.deleteRecursively(dirname)
-    reset() // drop all tables that were registered as part of the tests
-  }
-
-  // in case tests are failing we delete before and after each test
-  override def beforeEach() {
-    Utils.deleteRecursively(dirname)
-  }
+class HiveParquetSuite extends QueryTest with ParquetTest {
+  val sqlContext = TestHive
 
-  override def afterEach() {
-    Utils.deleteRecursively(dirname)
-  }
+  import sqlContext._
 
   test("Case insensitive attribute names") {
-    val tempFile = File.createTempFile("parquet", "")
-    tempFile.delete()
-    sparkContext.parallelize(1 to 10)
-      .map(_.toString)
-      .map(i => Cases(i, i))
-      .saveAsParquetFile(tempFile.getCanonicalPath)
-
-    parquetFile(tempFile.getCanonicalPath).registerTempTable("cases")
-    sql("SELECT upper FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString)
-    sql("SELECT LOWER FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString)
+    withParquetTable((1 to 4).map(i => Cases(i.toString, i.toString)), "cases") {
+      val expected = (1 to 4).map(i => Row(i.toString))
+      checkAnswer(sql("SELECT upper FROM cases"), expected)
+      checkAnswer(sql("SELECT LOWER FROM cases"), expected)
+    }
   }
 
   test("SELECT on Parquet table") {
-    val rdd = sql("SELECT * FROM testsource").collect()
-    assert(rdd != null)
-    assert(rdd.forall(_.size == 6))
+    val data = (1 to 4).map(i => (i, s"val_$i"))
+    withParquetTable(data, "t") {
+      checkAnswer(sql("SELECT * FROM t"), data)
+    }
   }
 
   test("Simple column projection + filter on Parquet table") {
-    val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect()
-    assert(rdd.size === 5, "Filter returned incorrect number of rows")
-    assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value")
+    withParquetTable((1 to 4).map(i => (i % 2 == 0, i, s"val_$i")), "t") {
+      checkAnswer(
+        sql("SELECT `_1`, `_3` FROM t WHERE `_1` = true"),
+        Seq(Row(true, "val_2"), Row(true, "val_4")))
+    }
   }
 
   test("Converting Hive to Parquet Table via saveAsParquetFile") {
-    sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath)
-    parquetFile(dirname.getAbsolutePath).registerTempTable("ptable")
-    val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0))
-    val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0))
-
-    compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String"))
+    withTempPath { dir =>
+      sql("SELECT * FROM src").saveAsParquetFile(dir.getCanonicalPath)
+      parquetFile(dir.getCanonicalPath).registerTempTable("p")
+      withTempTable("p") {
+        checkAnswer(
+          sql("SELECT * FROM src ORDER BY key"),
+          sql("SELECT * from p ORDER BY key").collect().toSeq)
+      }
+    }
   }
 
-  test("INSERT OVERWRITE TABLE Parquet table") {
-    sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath)
-    parquetFile(dirname.getAbsolutePath).registerTempTable("ptable")
-    // let's do three overwrites for good measure
-    sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
-    sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
-    sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
-    val rddCopy = sql("SELECT * FROM ptable").collect()
-    val rddOrig = sql("SELECT * FROM testsource").collect()
-    assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??")
-    compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames)
-  }
 
-  private def compareRDDs(rddOne: Array[Row], rddTwo: Array[Row], tableName: String, fieldNames: Seq[String]) {
-    var counter = 0
-    (rddOne, rddTwo).zipped.foreach {
-      (a,b) => (a,b).zipped.toArray.zipWithIndex.foreach {
-        case ((value_1, value_2), index) =>
-          assert(value_1 === value_2, s"table $tableName row $counter field ${fieldNames(index)} don't match")
+  test("INSERT OVERWRITE TABLE Parquet table") {
+    withParquetTable((1 to 4).map(i => (i, s"val_$i")), "t") {
+      withTempPath { file =>
+        sql("SELECT * FROM t LIMIT 1").saveAsParquetFile(file.getCanonicalPath)
+        parquetFile(file.getCanonicalPath).registerTempTable("p")
+        withTempTable("p") {
+          // let's do three overwrites for good measure
+          sql("INSERT OVERWRITE TABLE p SELECT * FROM t")
+          sql("INSERT OVERWRITE TABLE p SELECT * FROM t")
+          sql("INSERT OVERWRITE TABLE p SELECT * FROM t")
+          checkAnswer(sql("SELECT * FROM p"), sql("SELECT * FROM t").collect().toSeq)
+        }
       }
-    counter = counter + 1
     }
   }
 }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala
index 488ebba043794..fc0e42c201d56 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala
@@ -37,7 +37,7 @@ case class ParquetDataWithKey(p: Int, intField: Int, stringField: String)
  * A suite to test the automatic conversion of metastore tables with parquet data to use the
  * built in parquet support.
  */
-class ParquetMetastoreSuite extends ParquetTest {
+class ParquetMetastoreSuite extends ParquetPartitioningTest {
   override def beforeAll(): Unit = {
     super.beforeAll()
 
@@ -112,7 +112,7 @@ class ParquetMetastoreSuite extends ParquetTest {
 /**
  * A suite of tests for the Parquet support through the data sources API.
  */
-class ParquetSourceSuite extends ParquetTest {
+class ParquetSourceSuite extends ParquetPartitioningTest {
   override def beforeAll(): Unit = {
     super.beforeAll()
 
@@ -145,7 +145,7 @@ class ParquetSourceSuite extends ParquetTest {
 /**
  * A collection of tests for parquet data with various forms of partitioning.
  */
-abstract class ParquetTest extends QueryTest with BeforeAndAfterAll {
+abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll {
   var partitionedTableDir: File = null
   var partitionedTableDirWithKey: File = null
 
diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala
index 754ffc422072d..2d01a85067518 100644
--- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala
+++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector,
 import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory}
 import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils}
 import org.apache.hadoop.hive.serde2.{io => hiveIo}
+import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.{io => hadoopIo}
 import org.apache.hadoop.mapred.InputFormat
 import org.apache.spark.sql.catalyst.types.decimal.Decimal
@@ -71,76 +72,114 @@ private[hive] object HiveShim {
   def getStringWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.STRING,
-      if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]))
+      getStringWritable(value))
 
   def getIntWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.INT,
-      if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]))
+      getIntWritable(value))
 
   def getDoubleWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.DOUBLE,
-      if (value == null) null else new hiveIo.DoubleWritable(value.asInstanceOf[Double]))
+      getDoubleWritable(value))
 
   def getBooleanWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.BOOLEAN,
-      if (value == null) null else new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean]))
+      getBooleanWritable(value))
 
   def getLongWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.LONG,
-      if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long]))
+      getLongWritable(value))
 
   def getFloatWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.FLOAT,
-      if (value == null) null else new hadoopIo.FloatWritable(value.asInstanceOf[Float]))
+      getFloatWritable(value))
 
   def getShortWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.SHORT,
-      if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short]))
+      getShortWritable(value))
 
   def getByteWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.BYTE,
-      if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte]))
+      getByteWritable(value))
 
   def getBinaryWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.BINARY,
-      if (value == null) null else new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]))
+      getBinaryWritable(value))
 
   def getDateWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.DATE,
-      if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]))
+      getDateWritable(value))
 
   def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.TIMESTAMP,
-      if (value == null) {
-        null
-      } else {
-        new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp])
-      })
+      getTimestampWritable(value))
 
   def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.DECIMAL,
-      if (value == null) {
-        null
-      } else {
-        new hiveIo.HiveDecimalWritable(
-          HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying()))
-      })
+      getDecimalWritable(value))
 
   def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       PrimitiveCategory.VOID, null)
 
+  def getStringWritable(value: Any): hadoopIo.Text =
+    if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String])
+
+  def getIntWritable(value: Any): hadoopIo.IntWritable =
+    if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int])
+
+  def getDoubleWritable(value: Any): hiveIo.DoubleWritable =
+    if (value == null) null else new hiveIo.DoubleWritable(value.asInstanceOf[Double])
+
+  def getBooleanWritable(value: Any): hadoopIo.BooleanWritable =
+    if (value == null) null else new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean])
+
+  def getLongWritable(value: Any): hadoopIo.LongWritable =
+    if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long])
+
+  def getFloatWritable(value: Any): hadoopIo.FloatWritable =
+    if (value == null) null else new hadoopIo.FloatWritable(value.asInstanceOf[Float])
+
+  def getShortWritable(value: Any): hiveIo.ShortWritable =
+    if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short])
+
+  def getByteWritable(value: Any): hiveIo.ByteWritable =
+    if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte])
+
+  def getBinaryWritable(value: Any): hadoopIo.BytesWritable =
+    if (value == null) null else new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]])
+
+  def getDateWritable(value: Any): hiveIo.DateWritable =
+    if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date])
+
+  def getTimestampWritable(value: Any): hiveIo.TimestampWritable =
+    if (value == null) {
+      null
+    } else {
+      new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp])
+    }
+
+  def getDecimalWritable(value: Any): hiveIo.HiveDecimalWritable =
+    if (value == null) {
+      null
+    } else {
+      new hiveIo.HiveDecimalWritable(
+        HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying()))
+    }
+
+  def getPrimitiveNullWritable: NullWritable = NullWritable.get()
+
   def createDriverResultsArray = new JArrayList[String]
 
   def processResults(results: JArrayList[String]) = results
@@ -174,6 +213,7 @@ private[hive] object HiveShim {
 
   def compatibilityBlackList = Seq(
     "decimal_.*",
+    "udf7",
     "drop_partitions_filter2",
     "show_.*",
     "serde_regex",
@@ -196,7 +236,11 @@ private[hive] object HiveShim {
   }
 
   def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = {
-    Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue())
+    if (hdoi.preferWritable()) {
+      Decimal(hdoi.getPrimitiveWritableObject(data).getHiveDecimal().bigDecimalValue)
+    } else {
+      Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue())
+    }
   }
 }
 
diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
index 7c8cbf10c1c30..b78c75798e988 100644
--- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
+++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
@@ -22,6 +22,7 @@ import java.util.Properties
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.mapred.InputFormat
 import org.apache.hadoop.hive.common.StatsSetupConst
 import org.apache.hadoop.hive.common.`type`.{HiveDecimal}
@@ -163,91 +164,123 @@ private[hive] object HiveShim {
     new TableDesc(inputFormatClass, outputFormatClass, properties)
   }
 
+
   def getStringWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.stringTypeInfo,
-      if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]))
+      TypeInfoFactory.stringTypeInfo, getStringWritable(value))
 
   def getIntWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.intTypeInfo,
-      if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]))
+      TypeInfoFactory.intTypeInfo, getIntWritable(value))
 
   def getDoubleWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.doubleTypeInfo, if (value == null) {
-        null
-      } else {
-        new hiveIo.DoubleWritable(value.asInstanceOf[Double])
-      })
+      TypeInfoFactory.doubleTypeInfo, getDoubleWritable(value))
 
   def getBooleanWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.booleanTypeInfo, if (value == null) {
-        null
-      } else {
-        new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean])
-      })
+      TypeInfoFactory.booleanTypeInfo, getBooleanWritable(value))
 
   def getLongWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.longTypeInfo,
-      if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long]))
+      TypeInfoFactory.longTypeInfo, getLongWritable(value))
 
   def getFloatWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.floatTypeInfo, if (value == null) {
-        null
-      } else {
-        new hadoopIo.FloatWritable(value.asInstanceOf[Float])
-      })
+      TypeInfoFactory.floatTypeInfo, getFloatWritable(value))
 
   def getShortWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.shortTypeInfo,
-      if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short]))
+      TypeInfoFactory.shortTypeInfo, getShortWritable(value))
 
   def getByteWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.byteTypeInfo,
-      if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte]))
+      TypeInfoFactory.byteTypeInfo, getByteWritable(value))
 
   def getBinaryWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.binaryTypeInfo, if (value == null) {
-        null
-      } else {
-        new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]])
-      })
+      TypeInfoFactory.binaryTypeInfo, getBinaryWritable(value))
 
   def getDateWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.dateTypeInfo,
-      if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]))
+      TypeInfoFactory.dateTypeInfo, getDateWritable(value))
 
   def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.timestampTypeInfo, if (value == null) {
-        null
-      } else {
-        new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp])
-      })
+      TypeInfoFactory.timestampTypeInfo, getTimestampWritable(value))
 
   def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
-      TypeInfoFactory.decimalTypeInfo,
-      if (value == null) {
-        null
-      } else {
-        // TODO precise, scale?
-        new hiveIo.HiveDecimalWritable(
-          HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying()))
-      })
+      TypeInfoFactory.decimalTypeInfo, getDecimalWritable(value))
 
   def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector =
     PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
       TypeInfoFactory.voidTypeInfo, null)
 
+  def getStringWritable(value: Any): hadoopIo.Text =
+    if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String])
+
+  def getIntWritable(value: Any): hadoopIo.IntWritable =
+    if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int])
+
+  def getDoubleWritable(value: Any): hiveIo.DoubleWritable =
+    if (value == null) {
+      null
+    } else {
+      new hiveIo.DoubleWritable(value.asInstanceOf[Double])
+    }
+
+  def getBooleanWritable(value: Any): hadoopIo.BooleanWritable =
+    if (value == null) {
+      null
+    } else {
+      new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean])
+    }
+
+  def getLongWritable(value: Any): hadoopIo.LongWritable =
+    if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long])
+
+  def getFloatWritable(value: Any): hadoopIo.FloatWritable =
+    if (value == null) {
+      null
+    } else {
+      new hadoopIo.FloatWritable(value.asInstanceOf[Float])
+    }
+
+  def getShortWritable(value: Any): hiveIo.ShortWritable =
+    if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short])
+
+  def getByteWritable(value: Any): hiveIo.ByteWritable =
+    if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte])
+
+  def getBinaryWritable(value: Any): hadoopIo.BytesWritable =
+    if (value == null) {
+      null
+    } else {
+      new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]])
+    }
+
+  def getDateWritable(value: Any): hiveIo.DateWritable =
+    if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date])
+
+  def getTimestampWritable(value: Any): hiveIo.TimestampWritable =
+    if (value == null) {
+      null
+    } else {
+      new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp])
+    }
+
+  def getDecimalWritable(value: Any): hiveIo.HiveDecimalWritable =
+    if (value == null) {
+      null
+    } else {
+      // TODO precise, scale?
+      new hiveIo.HiveDecimalWritable(
+        HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying()))
+    }
+
+  def getPrimitiveNullWritable: NullWritable = NullWritable.get()
+
   def createDriverResultsArray = new JArrayList[Object]
 
   def processResults(results: JArrayList[Object]) = {
@@ -355,7 +388,12 @@ private[hive] object HiveShim {
   }
 
   def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = {
-    Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale())
+    if (hdoi.preferWritable()) {
+      Decimal(hdoi.getPrimitiveWritableObject(data).getHiveDecimal().bigDecimalValue,
+        hdoi.precision(), hdoi.scale())
+    } else {
+      Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale())
+    }
   }
 }
 
diff --git a/streaming/pom.xml b/streaming/pom.xml
index b8b8f2e6cab65..d3c6d0347a622 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -48,11 +48,6 @@
       org.scala-lang
       scala-library
     
-    
-      org.scalatest
-      scalatest_${scala.binary.version}
-      test
-    
     
       org.scalacheck
       scalacheck_${scala.binary.version}
@@ -73,18 +68,13 @@
     target/scala-${scala.binary.version}/classes
     target/scala-${scala.binary.version}/test-classes
     
-      
-        org.scalatest
-        scalatest-maven-plugin
-      
-      
-      
       
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala
index a0aeacbc733bd..fdbbe2aa6ef08 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala
@@ -17,30 +17,63 @@
 
 package org.apache.spark.streaming
 
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.locks.ReentrantLock
+
 private[streaming] class ContextWaiter {
+
+  private val lock = new ReentrantLock()
+  private val condition = lock.newCondition()
+
+  // Guarded by "lock"
   private var error: Throwable = null
-  private var stopped: Boolean = false
 
-  def notifyError(e: Throwable) = synchronized {
-    error = e
-    notifyAll()
-  }
+  // Guarded by "lock"
+  private var stopped: Boolean = false
 
-  def notifyStop() = synchronized {
-    stopped = true
-    notifyAll()
+  def notifyError(e: Throwable): Unit = {
+    lock.lock()
+    try {
+      error = e
+      condition.signalAll()
+    } finally {
+      lock.unlock()
+    }
   }
 
-  def waitForStopOrError(timeout: Long = -1) = synchronized {
-    // If already had error, then throw it
-    if (error != null) {
-      throw error
+  def notifyStop(): Unit = {
+    lock.lock()
+    try {
+      stopped = true
+      condition.signalAll()
+    } finally {
+      lock.unlock()
     }
+  }
 
-    // If not already stopped, then wait
-    if (!stopped) {
-      if (timeout < 0) wait() else wait(timeout)
+  /**
+   * Return `true` if it's stopped; or throw the reported error if `notifyError` has been called; or
+   * `false` if the waiting time detectably elapsed before return from the method.
+   */
+  def waitForStopOrError(timeout: Long = -1): Boolean = {
+    lock.lock()
+    try {
+      if (timeout < 0) {
+        while (!stopped && error == null) {
+          condition.await()
+        }
+      } else {
+        var nanos = TimeUnit.MILLISECONDS.toNanos(timeout)
+        while (!stopped && error == null && nanos > 0) {
+          nanos = condition.awaitNanos(nanos)
+        }
+      }
+      // If already had error, then throw it
       if (error != null) throw error
+      // already stopped or timeout
+      stopped
+    } finally {
+      lock.unlock()
     }
   }
 }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index ecab5510a8e7b..8ef0787137845 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger
 
 import scala.collection.Map
 import scala.collection.mutable.Queue
-import scala.language.implicitConversions
 import scala.reflect.ClassTag
 
 import akka.actor.{Props, SupervisorStrategy}
@@ -523,9 +522,11 @@ object StreamingContext extends Logging {
 
   private[streaming] val DEFAULT_CLEANER_TTL = 3600
 
-  implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)])
+  @deprecated("Replaced by implicit functions in the DStream companion object. This is " +
+    "kept here only for backward compatibility.", "1.3.0")
+  def toPairDStreamFunctions[K, V](stream: DStream[(K, V)])
       (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = {
-    new PairDStreamFunctions[K, V](stream)
+    DStream.toPairDStreamFunctions(stream)(kt, vt, ord)
   }
 
   /**
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala
index e35a568ddf115..9697437dd2fe5 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala
@@ -29,9 +29,17 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source {
   private val streamingListener = ssc.progressListener
 
   private def registerGauge[T](name: String, f: StreamingJobProgressListener => T,
-      defaultValue: T) {
+      defaultValue: T): Unit = {
+    registerGaugeWithOption[T](name,
+      (l: StreamingJobProgressListener) => Option(f(streamingListener)), defaultValue)
+  }
+
+  private def registerGaugeWithOption[T](
+      name: String,
+      f: StreamingJobProgressListener => Option[T],
+      defaultValue: T): Unit = {
     metricRegistry.register(MetricRegistry.name("streaming", name), new Gauge[T] {
-      override def getValue: T = Option(f(streamingListener)).getOrElse(defaultValue)
+      override def getValue: T = f(streamingListener).getOrElse(defaultValue)
     })
   }
 
@@ -41,6 +49,12 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source {
   // Gauge for number of total completed batches
   registerGauge("totalCompletedBatches", _.numTotalCompletedBatches, 0L)
 
+  // Gauge for number of total received records
+  registerGauge("totalReceivedRecords", _.numTotalReceivedRecords, 0L)
+
+  // Gauge for number of total processed records
+  registerGauge("totalProcessedRecords", _.numTotalProcessedRecords, 0L)
+
   // Gauge for number of unprocessed batches
   registerGauge("unprocessedBatches", _.numUnprocessedBatches, 0L)
 
@@ -55,19 +69,30 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source {
 
   // Gauge for last completed batch, useful for monitoring the streaming job's running status,
   // displayed data -1 for any abnormal condition.
-  registerGauge("lastCompletedBatch_submissionTime",
-    _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L)
-  registerGauge("lastCompletedBatch_processStartTime",
-    _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L)
-  registerGauge("lastCompletedBatch_processEndTime",
-    _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L)
+  registerGaugeWithOption("lastCompletedBatch_submissionTime",
+    _.lastCompletedBatch.map(_.submissionTime), -1L)
+  registerGaugeWithOption("lastCompletedBatch_processingStartTime",
+    _.lastCompletedBatch.flatMap(_.processingStartTime), -1L)
+  registerGaugeWithOption("lastCompletedBatch_processingEndTime",
+    _.lastCompletedBatch.flatMap(_.processingEndTime), -1L)
+
+  // Gauge for last completed batch's delay information.
+  registerGaugeWithOption("lastCompletedBatch_processingDelay",
+    _.lastCompletedBatch.flatMap(_.processingDelay), -1L)
+  registerGaugeWithOption("lastCompletedBatch_schedulingDelay",
+    _.lastCompletedBatch.flatMap(_.schedulingDelay), -1L)
+  registerGaugeWithOption("lastCompletedBatch_totalDelay",
+    _.lastCompletedBatch.flatMap(_.totalDelay), -1L)
 
   // Gauge for last received batch, useful for monitoring the streaming job's running status,
   // displayed data -1 for any abnormal condition.
-  registerGauge("lastReceivedBatch_submissionTime",
-    _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L)
-  registerGauge("lastReceivedBatch_processStartTime",
-    _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L)
-  registerGauge("lastReceivedBatch_processEndTime",
-    _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L)
+  registerGaugeWithOption("lastReceivedBatch_submissionTime",
+    _.lastCompletedBatch.map(_.submissionTime), -1L)
+  registerGaugeWithOption("lastReceivedBatch_processingStartTime",
+    _.lastCompletedBatch.flatMap(_.processingStartTime), -1L)
+  registerGaugeWithOption("lastReceivedBatch_processingEndTime",
+    _.lastCompletedBatch.flatMap(_.processingEndTime), -1L)
+
+  // Gauge for last received batch records.
+  registerGauge("lastReceivedBatch_records", _.lastReceivedBatchRecords.values.sum, 0L)
 }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
index 2a7004e56ef53..e0542eda1383f 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
@@ -51,7 +51,15 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
    * operator, so this DStream will be registered as an output stream and there materialized.
    */
   def print(): Unit = {
-    dstream.print()
+    print(10)
+  }
+
+  /**
+   * Print the first num elements of each RDD generated in this DStream. This is an output
+   * operator, so this DStream will be registered as an output stream and there materialized.
+   */
+  def print(num: Int): Unit = {
+    dstream.print(num)
   }
 
   /**
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
index bb44b906d7386..de124cf40eff1 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
@@ -36,7 +36,6 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J
 import org.apache.spark.rdd.RDD
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming._
-import org.apache.spark.streaming.StreamingContext._
 import org.apache.spark.streaming.dstream.DStream
 
 /**
@@ -815,6 +814,6 @@ object JavaPairDStream {
 
   def scalaToJavaLong[K: ClassTag](dstream: JavaPairDStream[K, Long])
   : JavaPairDStream[K, JLong] = {
-    StreamingContext.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_))
+    DStream.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_))
   }
 }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
index dbf1ebbaf653a..b874f561c12eb 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
@@ -20,16 +20,16 @@ package org.apache.spark.streaming.dstream
 
 import java.io.{IOException, ObjectInputStream, ObjectOutputStream}
 
-import scala.deprecated
 import scala.collection.mutable.HashMap
+import scala.language.implicitConversions
 import scala.reflect.ClassTag
 import scala.util.matching.Regex
 
 import org.apache.spark.{Logging, SparkException}
-import org.apache.spark.rdd.{BlockRDD, RDD}
+import org.apache.spark.rdd.{BlockRDD, PairRDDFunctions, RDD}
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming._
-import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.StreamingContext.rddToFileName
 import org.apache.spark.streaming.scheduler.Job
 import org.apache.spark.util.{CallSite, MetadataCleaner, Utils}
 
@@ -48,8 +48,7 @@ import org.apache.spark.util.{CallSite, MetadataCleaner, Utils}
  * `window`. In addition, [[org.apache.spark.streaming.dstream.PairDStreamFunctions]] contains
  * operations available only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and
  * `join`. These operations are automatically available on any DStream of pairs
- * (e.g., DStream[(Int, Int)] through implicit conversions when
- * `org.apache.spark.streaming.StreamingContext._` is imported.
+ * (e.g., DStream[(Int, Int)] through implicit conversions.
  *
  * DStreams internally is characterized by a few basic properties:
  *  - A list of other DStreams that the DStream depends on
@@ -293,7 +292,13 @@ abstract class DStream[T: ClassTag] (
         // set this DStream's creation site, generate RDDs and then restore the previous call site.
         val prevCallSite = ssc.sparkContext.getCallSite()
         ssc.sparkContext.setCallSite(creationSite)
-        val rddOption = compute(time)
+        // Disable checks for existing output directories in jobs launched by the streaming
+        // scheduler, since we may need to write output to an existing directory during checkpoint
+        // recovery; see SPARK-4835 for more details. We need to have this call here because
+        // compute() might cause Spark jobs to be launched.
+        val rddOption = PairRDDFunctions.disableOutputSpecValidation.withValue(true) {
+          compute(time)
+        }
         ssc.sparkContext.setCallSite(prevCallSite)
 
         rddOption.foreach { case newRDD =>
@@ -606,13 +611,21 @@ abstract class DStream[T: ClassTag] (
    * operator, so this DStream will be registered as an output stream and there materialized.
    */
   def print() {
+    print(10)
+  }
+
+  /**
+   * Print the first num elements of each RDD generated in this DStream. This is an output
+   * operator, so this DStream will be registered as an output stream and there materialized.
+   */
+  def print(num: Int) {
     def foreachFunc = (rdd: RDD[T], time: Time) => {
-      val first11 = rdd.take(11)
+      val firstNum = rdd.take(num + 1)
       println ("-------------------------------------------")
       println ("Time: " + time)
       println ("-------------------------------------------")
-      first11.take(10).foreach(println)
-      if (first11.size > 10) println("...")
+      firstNum.take(num).foreach(println)
+      if (firstNum.size > num) println("...")
       println()
     }
     new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register()
@@ -802,10 +815,21 @@ abstract class DStream[T: ClassTag] (
   }
 }
 
-private[streaming] object DStream {
+object DStream {
+
+  // `toPairDStreamFunctions` was in SparkContext before 1.3 and users had to
+  // `import StreamingContext._` to enable it. Now we move it here to make the compiler find
+  // it automatically. However, we still keep the old function in StreamingContext for backward
+  // compatibility and forward to the following function directly.
+
+  implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)])
+      (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null):
+    PairDStreamFunctions[K, V] = {
+    new PairDStreamFunctions[K, V](stream)
+  }
 
   /** Get the creation site of a DStream from the stack trace of when the DStream is created. */
-  def getCreationSite(): CallSite = {
+  private[streaming] def getCreationSite(): CallSite = {
     val SPARK_CLASS_REGEX = """^org\.apache\.spark""".r
     val SPARK_STREAMING_TESTCLASS_REGEX = """^org\.apache\.spark\.streaming\.test""".r
     val SPARK_EXAMPLES_CLASS_REGEX = """^org\.apache\.spark\.examples""".r
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
index 5f13fdc5579ed..e7c5639a63499 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.streaming.dstream
 
 import java.io.{IOException, ObjectInputStream}
+import java.util.concurrent.ConcurrentHashMap
 
 import scala.collection.mutable
 import scala.reflect.ClassTag
@@ -74,12 +75,15 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
     newFilesOnly: Boolean = true)
   extends InputDStream[(K, V)](ssc_) {
 
+  // This is a def so that it works during checkpoint recovery:
+  private def clock = ssc.scheduler.clock
+
   // Data to be saved as part of the streaming checkpoints
   protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData
 
   // Initial ignore threshold based on which old, existing files in the directory (at the time of
   // starting the streaming application) will be ignored or considered
-  private val initialModTimeIgnoreThreshold = if (newFilesOnly) System.currentTimeMillis() else 0L
+  private val initialModTimeIgnoreThreshold = if (newFilesOnly) clock.currentTime() else 0L
 
   /*
    * Make sure that the information of files selected in the last few batches are remembered.
@@ -91,8 +95,9 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
   remember(durationToRemember)
 
   // Map of batch-time to selected file info for the remembered batches
+  // This is a concurrent map because it's also accessed in unit tests
   @transient private[streaming] var batchTimeToSelectedFiles =
-    new mutable.HashMap[Time, Array[String]]
+    new mutable.HashMap[Time, Array[String]] with mutable.SynchronizedMap[Time, Array[String]]
 
   // Set of files that were selected in the remembered batches
   @transient private var recentlySelectedFiles = new mutable.HashSet[String]()
@@ -151,7 +156,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
    */
   private def findNewFiles(currentTime: Long): Array[String] = {
     try {
-      lastNewFileFindingTime = System.currentTimeMillis
+      lastNewFileFindingTime = clock.currentTime()
 
       // Calculate ignore threshold
       val modTimeIgnoreThreshold = math.max(
@@ -164,7 +169,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
         def accept(path: Path): Boolean = isNewFile(path, currentTime, modTimeIgnoreThreshold)
       }
       val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString)
-      val timeTaken = System.currentTimeMillis - lastNewFileFindingTime
+      val timeTaken = clock.currentTime() - lastNewFileFindingTime
       logInfo("Finding new files took " + timeTaken + " ms")
       logDebug("# cached file times = " + fileToModTime.size)
       if (timeTaken > slideDuration.milliseconds) {
@@ -267,7 +272,8 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
     logDebug(this.getClass().getSimpleName + ".readObject used")
     ois.defaultReadObject()
     generatedRDDs = new mutable.HashMap[Time, RDD[(K,V)]] ()
-    batchTimeToSelectedFiles = new mutable.HashMap[Time, Array[String]]()
+    batchTimeToSelectedFiles =
+      new mutable.HashMap[Time, Array[String]] with mutable.SynchronizedMap[Time, Array[String]]
     recentlySelectedFiles = new mutable.HashSet[String]()
     fileToModTime = new TimeStampedHashMap[String, Long](true)
   }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala
index 98539e06b4e29..8a58571632447 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala
@@ -27,12 +27,10 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
 import org.apache.spark.{HashPartitioner, Partitioner, SerializableWritable}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.streaming.{Duration, Time}
-import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.StreamingContext.rddToFileName
 
 /**
  * Extra functions available on DStream of (key, value) pairs through an implicit conversion.
- * Import `org.apache.spark.streaming.StreamingContext._` at the top of your program to use
- * these functions.
  */
 class PairDStreamFunctions[K, V](self: DStream[(K,V)])
     (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K])
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
index 1a47089e513c4..c0a5af0b65cc3 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
@@ -17,8 +17,6 @@
 
 package org.apache.spark.streaming.dstream
 
-import org.apache.spark.streaming.StreamingContext._
-
 import org.apache.spark.rdd.RDD
 import org.apache.spark.rdd.{CoGroupedRDD, MapPartitionsRDD}
 import org.apache.spark.Partitioner
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
index 7cd4554282ca1..71b61856e23c0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.streaming.dstream
 
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{PairRDDFunctions, RDD}
 import org.apache.spark.streaming.{Duration, Time}
 import scala.reflect.ClassTag
 
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/package.scala b/streaming/src/main/scala/org/apache/spark/streaming/package.scala
index 4dd985cf5a178..2153ae0d34184 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/package.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/package.scala
@@ -26,7 +26,7 @@ package org.apache.spark
  * available only on DStreams
  * of key-value pairs, such as `groupByKey` and `reduceByKey`. These operations are automatically
  * available on any DStream of the right type (e.g. DStream[(Int, Int)] through implicit
- * conversions when you `import org.apache.spark.streaming.StreamingContext._`.
+ * conversions.
  *
  * For the Java API of Spark Streaming, take a look at the
  * [[org.apache.spark.streaming.api.java.JavaStreamingContext]] which serves as the entry point, and
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala
index 1868a1ebc7b4a..a7d63bd4f2dbf 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala
@@ -123,7 +123,7 @@ private[streaming] case class ByteBufferData(bytes: ByteBuffer) extends ActorRec
  * As Actors can also be used to receive data from almost any stream source.
  * A nice set of abstraction(s) for actors as receivers is already provided for
  * a few general cases. It is thus exposed as an API where user may come with
- * his own Actor to run as receiver for Spark Streaming input source.
+ * their own Actor to run as receiver for Spark Streaming input source.
  *
  * This starts a supervisor actor which starts workers and also provides
  * [http://doc.akka.io/docs/akka/snapshot/scala/fault-tolerance.html fault-tolerance].
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala
index 55765dc90698b..79263a7183977 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala
@@ -116,7 +116,7 @@ private[streaming] class BlockGenerator(
 
   /**
    * Push a single data item into the buffer. After buffering the data, the
-   * `BlockGeneratorListnere.onAddData` callback will be called. All received data items
+   * `BlockGeneratorListener.onAddData` callback will be called. All received data items
    * will be periodically pushed into BlockManager.
    */
   def addDataWithCallback(data: Any, metadata: Any) = synchronized {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala
index fdf995320beb4..f7a8ebee8a544 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala
@@ -42,7 +42,7 @@ private[streaming] trait ReceivedBlockHandler {
   def storeBlock(blockId: StreamBlockId, receivedBlock: ReceivedBlock): ReceivedBlockStoreResult
 
   /** Cleanup old blocks older than the given threshold time */
-  def cleanupOldBlock(threshTime: Long)
+  def cleanupOldBlocks(threshTime: Long)
 }
 
 
@@ -82,7 +82,7 @@ private[streaming] class BlockManagerBasedBlockHandler(
     BlockManagerBasedStoreResult(blockId)
   }
 
-  def cleanupOldBlock(threshTime: Long) {
+  def cleanupOldBlocks(threshTime: Long) {
     // this is not used as blocks inserted into the BlockManager are cleared by DStream's clearing
     // of BlockRDDs.
   }
@@ -121,6 +121,24 @@ private[streaming] class WriteAheadLogBasedBlockHandler(
   private val maxFailures = conf.getInt(
     "spark.streaming.receiver.writeAheadLog.maxFailures", 3)
 
+  private val effectiveStorageLevel = {
+    if (storageLevel.deserialized) {
+      logWarning(s"Storage level serialization ${storageLevel.deserialized} is not supported when" +
+        s" write ahead log is enabled, change to serialization false")
+    }
+    if (storageLevel.replication > 1) {
+      logWarning(s"Storage level replication ${storageLevel.replication} is unnecessary when " +
+        s"write ahead log is enabled, change to replication 1")
+    }
+
+    StorageLevel(storageLevel.useDisk, storageLevel.useMemory, storageLevel.useOffHeap, false, 1)
+  }
+
+  if (storageLevel != effectiveStorageLevel) {
+    logWarning(s"User defined storage level $storageLevel is changed to effective storage level " +
+      s"$effectiveStorageLevel when write ahead log is enabled")
+  }
+
   // Manages rolling log files
   private val logManager = new WriteAheadLogManager(
     checkpointDirToLogDir(checkpointDir, streamId),
@@ -156,7 +174,7 @@ private[streaming] class WriteAheadLogBasedBlockHandler(
     // Store the block in block manager
     val storeInBlockManagerFuture = Future {
       val putResult =
-        blockManager.putBytes(blockId, serializedBlock, storageLevel, tellMaster = true)
+        blockManager.putBytes(blockId, serializedBlock, effectiveStorageLevel, tellMaster = true)
       if (!putResult.map { _._1 }.contains(blockId)) {
         throw new SparkException(
           s"Could not store $blockId to block manager with storage level $storageLevel")
@@ -169,16 +187,13 @@ private[streaming] class WriteAheadLogBasedBlockHandler(
     }
 
     // Combine the futures, wait for both to complete, and return the write ahead log segment
-    val combinedFuture = for {
-      _ <- storeInBlockManagerFuture
-      fileSegment <- storeInWriteAheadLogFuture
-    } yield fileSegment
+    val combinedFuture = storeInBlockManagerFuture.zip(storeInWriteAheadLogFuture).map(_._2)
     val segment = Await.result(combinedFuture, blockStoreTimeout)
     WriteAheadLogBasedStoreResult(blockId, segment)
   }
 
-  def cleanupOldBlock(threshTime: Long) {
-    logManager.cleanupOldLogs(threshTime)
+  def cleanupOldBlocks(threshTime: Long) {
+    logManager.cleanupOldLogs(threshTime, waitForCompletion = false)
   }
 
   def stop() {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala
index bf39d1e891cae..ab9fa192191aa 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala
@@ -18,6 +18,6 @@
 package org.apache.spark.streaming.receiver
 
 /** Messages sent to the NetworkReceiver. */
-private[streaming] sealed trait ReceiverMessage
+private[streaming] sealed trait ReceiverMessage extends Serializable
 private[streaming] object StopReceiver extends ReceiverMessage
 
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
index cfa3cd8925c80..0e0f5bd3b9db4 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
@@ -22,6 +22,7 @@ import scala.collection.JavaConversions._
 import java.util.concurrent.{TimeUnit, ConcurrentHashMap, Executors}
 import akka.actor.{ActorRef, Actor, Props}
 import org.apache.spark.{SparkException, Logging, SparkEnv}
+import org.apache.spark.rdd.PairRDDFunctions
 import org.apache.spark.streaming._
 
 
@@ -168,7 +169,12 @@ class JobScheduler(val ssc: StreamingContext) extends Logging {
   private class JobHandler(job: Job) extends Runnable {
     def run() {
       eventActor ! JobStarted(job)
-      job.run()
+      // Disable checks for existing output directories in jobs launched by the streaming scheduler,
+      // since we may need to write output to an existing directory during checkpoint recovery;
+      // see SPARK-4835 for more details.
+      PairRDDFunctions.disableOutputSpecValidation.withValue(true) {
+        job.run()
+      }
       eventActor ! JobCompleted(job)
     }
   }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala
index 02758e0bca6c5..2ce458cddec1a 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala
@@ -139,14 +139,17 @@ private[streaming] class ReceivedBlockTracker(
     getReceivedBlockQueue(streamId).toSeq
   }
 
-  /** Clean up block information of old batches. */
-  def cleanupOldBatches(cleanupThreshTime: Time): Unit = synchronized {
+  /**
+   * Clean up block information of old batches. If waitForCompletion is true, this method
+   * returns only after the files are cleaned up.
+   */
+  def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized {
     assert(cleanupThreshTime.milliseconds < clock.currentTime())
     val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq
     logInfo("Deleting batches " + timesToCleanup)
     writeToLog(BatchCleanupEvent(timesToCleanup))
     timeToAllocatedBlocks --= timesToCleanup
-    logManagerOption.foreach(_.cleanupOldLogs(cleanupThreshTime.milliseconds))
+    logManagerOption.foreach(_.cleanupOldLogs(cleanupThreshTime.milliseconds, waitForCompletion))
     log
   }
 
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
index 32e481dabc8ca..8dbb42a86e3bd 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
@@ -121,7 +121,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
 
     /** Clean up metadata older than the given threshold time */
   def cleanupOldMetadata(cleanupThreshTime: Time) {
-    receivedBlockTracker.cleanupOldBatches(cleanupThreshTime)
+    receivedBlockTracker.cleanupOldBatches(cleanupThreshTime, waitForCompletion = false)
   }
 
   /** Register a receiver */
@@ -150,8 +150,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
         logWarning("No prior receiver info")
         ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error)
     }
-    receiverInfo(streamId) = newReceiverInfo
-    listenerBus.post(StreamingListenerReceiverStopped(receiverInfo(streamId)))
+    receiverInfo -= streamId
+    listenerBus.post(StreamingListenerReceiverStopped(newReceiverInfo))
     val messageWithError = if (error != null && !error.isEmpty) {
       s"$message - $error"
     } else {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala
index f61069b56db5e..5ee53a5c5f561 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala
@@ -25,7 +25,6 @@ import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted
 import org.apache.spark.streaming.scheduler.BatchInfo
 import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted
 import org.apache.spark.util.Distribution
-import org.apache.spark.Logging
 
 
 private[streaming] class StreamingJobProgressListener(ssc: StreamingContext)
@@ -36,6 +35,8 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext)
   private val completedaBatchInfos = new Queue[BatchInfo]
   private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100)
   private var totalCompletedBatches = 0L
+  private var totalReceivedRecords = 0L
+  private var totalProcessedRecords = 0L
   private val receiverInfos = new HashMap[Int, ReceiverInfo]
 
   val batchDuration = ssc.graph.batchDuration.milliseconds
@@ -65,6 +66,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext)
   override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) = synchronized {
     runningBatchInfos(batchStarted.batchInfo.batchTime) = batchStarted.batchInfo
     waitingBatchInfos.remove(batchStarted.batchInfo.batchTime)
+
+    batchStarted.batchInfo.receivedBlockInfo.foreach { case (_, infos) =>
+      totalReceivedRecords += infos.map(_.numRecords).sum
+    }
   }
 
   override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) = synchronized {
@@ -73,6 +78,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext)
     completedaBatchInfos.enqueue(batchCompleted.batchInfo)
     if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue()
     totalCompletedBatches += 1L
+
+    batchCompleted.batchInfo.receivedBlockInfo.foreach { case (_, infos) =>
+      totalProcessedRecords += infos.map(_.numRecords).sum
+    }
   }
 
   def numReceivers = synchronized {
@@ -83,6 +92,14 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext)
     totalCompletedBatches
   }
 
+  def numTotalReceivedRecords: Long = synchronized {
+    totalReceivedRecords
+  }
+
+  def numTotalProcessedRecords: Long = synchronized {
+    totalProcessedRecords
+  }
+
   def numUnprocessedBatches: Long = synchronized {
     waitingBatchInfos.size + runningBatchInfos.size
   }
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
index 1353e487c72cf..98e9a2e639e25 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
@@ -67,6 +67,12 @@ private[ui] class StreamingPage(parent: StreamingTab)
       
  • Waiting batches: {listener.numUnprocessedBatches}
  • +
  • + Received records: {listener.numTotalReceivedRecords} +
  • +
  • + Processed records: {listener.numTotalProcessedRecords} +
  • } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index 7cd867ce34b87..d6d96d7ba00fd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -59,9 +59,11 @@ class SystemClock() extends Clock { private[streaming] class ManualClock() extends Clock { - var time = 0L + private var time = 0L - def currentTime() = time + def currentTime() = this.synchronized { + time + } def setTime(timeToSet: Long) = { this.synchronized { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala index 70d234320be7c..166661b7496df 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala @@ -19,11 +19,11 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, ExecutionContext, Future} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.Logging import org.apache.spark.util.Utils import WriteAheadLogManager._ @@ -124,8 +124,12 @@ private[streaming] class WriteAheadLogManager( * files, which is usually based on the local system time. So if there is coordination necessary * between the node calculating the threshTime (say, driver node), and the local system time * (say, worker node), the caller has to take account of possible time skew. + * + * If waitForCompletion is set to true, this method will return only after old logs have been + * deleted. This should be set to true only for testing. Else the files will be deleted + * asynchronously. */ - def cleanupOldLogs(threshTime: Long): Unit = { + def cleanupOldLogs(threshTime: Long, waitForCompletion: Boolean): Unit = { val oldLogFiles = synchronized { pastLogs.filter { _.endTime < threshTime } } logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " + s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}") @@ -146,10 +150,15 @@ private[streaming] class WriteAheadLogManager( logInfo(s"Cleared log files in $logDirectory older than $threshTime") } if (!executionContext.isShutdown) { - Future { deleteFiles() } + val f = Future { deleteFiles() } + if (waitForCompletion) { + import scala.concurrent.duration._ + Await.ready(f, 1 second) + } } } + /** Stop the manager, close any open log writer */ def stop(): Unit = synchronized { if (currentLogWriter != null) { diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index ce645fccba1d0..12cc0de7509d6 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -57,7 +57,7 @@ public void equalIterable(Iterable a, Iterable b) { @Test public void testInitialization() { - Assert.assertNotNull(ssc.sc()); + Assert.assertNotNull(ssc.sparkContext()); } @SuppressWarnings("unchecked") @@ -662,7 +662,7 @@ public void testStreamingContextTransform(){ listOfDStreams1, new Function2>, Time, JavaRDD>() { public JavaRDD call(List> listOfRDDs, Time time) { - assert(listOfRDDs.size() == 2); + Assert.assertEquals(2, listOfRDDs.size()); return null; } } @@ -675,7 +675,7 @@ public JavaRDD call(List> listOfRDDs, Time time) { listOfDStreams2, new Function2>, Time, JavaPairRDD>>() { public JavaPairRDD> call(List> listOfRDDs, Time time) { - assert(listOfRDDs.size() == 3); + Assert.assertEquals(3, listOfRDDs.size()); JavaRDD rdd1 = (JavaRDD)listOfRDDs.get(0); JavaRDD rdd2 = (JavaRDD)listOfRDDs.get(1); JavaRDD> rdd3 = (JavaRDD>)listOfRDDs.get(2); @@ -969,7 +969,7 @@ public Integer call(Tuple2 in) throws Exception { }); JavaTestUtils.attachTestOutputStream(reversed); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } @@ -1012,7 +1012,7 @@ public Iterable> call(Tuple2 in) throws } }); JavaTestUtils.attachTestOutputStream(flatMapped); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } @@ -1163,9 +1163,9 @@ public void testGroupByKeyAndWindow() { JavaTestUtils.attachTestOutputStream(groupWindowed); List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); - assert(result.size() == expected.size()); + Assert.assertEquals(expected.size(), result.size()); for (int i = 0; i < result.size(); i++) { - assert(convert(result.get(i)).equals(convert(expected.get(i)))); + Assert.assertEquals(convert(expected.get(i)), convert(result.get(i))); } } @@ -1383,7 +1383,7 @@ public JavaPairRDD call(JavaPairRDD in) thro }); JavaTestUtils.attachTestOutputStream(sorted); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } diff --git a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 4411d6e20c52a..9697237bfa1a3 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/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=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/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 86b96785d7b87..e8f4a7779ec21 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -28,7 +28,6 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.SparkContext._ import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.{DStream, WindowedDStream} import org.apache.spark.HashPartitioner @@ -639,7 +638,7 @@ class BasicOperationsSuite extends TestSuiteBase { if (rememberDuration != null) ssc.remember(rememberDuration) val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput) val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - assert(clock.time === Seconds(10).milliseconds) + assert(clock.currentTime() === Seconds(10).milliseconds) assert(output.size === numExpectedOutput) operatedStream } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index c97998add8ffa..8f8bc61437ba5 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -18,19 +18,19 @@ package org.apache.spark.streaming import java.io.File -import java.nio.charset.Charset -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.reflect.ClassTag +import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{IntWritable, Text} import org.apache.hadoop.mapred.TextOutputFormat import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} +import org.scalatest.concurrent.Eventually._ -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.{DStream, FileInputDStream} import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils @@ -46,8 +46,6 @@ class CheckpointSuite extends TestSuiteBase { override def batchDuration = Milliseconds(500) - override def actuallyWait = true // to allow checkpoints to be written - override def beforeFunction() { super.beforeFunction() Utils.deleteRecursively(new File(checkpointDir)) @@ -144,7 +142,6 @@ class CheckpointSuite extends TestSuiteBase { ssc.start() advanceTimeWithRealDelay(ssc, 4) ssc.stop() - System.clearProperty("spark.streaming.manualClock.jump") ssc = null } @@ -256,6 +253,45 @@ class CheckpointSuite extends TestSuiteBase { } } + test("recovery with saveAsHadoopFile inside transform operation") { + // Regression test for SPARK-4835. + // + // In that issue, the problem was that `saveAsHadoopFile(s)` would fail when the last batch + // was restarted from a checkpoint since the output directory would already exist. However, + // the other saveAsHadoopFile* tests couldn't catch this because they only tested whether the + // output matched correctly and not whether the post-restart batch had successfully finished + // without throwing any errors. The following test reproduces the same bug with a test that + // actually fails because the error in saveAsHadoopFile causes transform() to fail, which + // prevents the expected output from being written to the output stream. + // + // This is not actually a valid use of transform, but it's being used here so that we can test + // the fix for SPARK-4835 independently of additional test cleanup. + // + // After SPARK-5079 is addressed, should be able to remove this test since a strengthened + // version of the other saveAsHadoopFile* tests would prevent regressions for this issue. + val tempDir = Files.createTempDir() + try { + testCheckpointedOperation( + Seq(Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq()), + (s: DStream[String]) => { + s.transform { (rdd, time) => + val output = rdd.map(x => (x, 1)).reduceByKey(_ + _) + output.saveAsHadoopFile( + new File(tempDir, "result-" + time.milliseconds).getAbsolutePath, + classOf[Text], + classOf[IntWritable], + classOf[TextOutputFormat[Text, IntWritable]]) + output + } + }, + Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + 3 + ) + } finally { + Utils.deleteRecursively(tempDir) + } + } + // This tests whether the StateDStream's RDD checkpoints works correctly such // that the system can recover from a master failure. This assumes as reliable, // replayable input source - TestInputDStream. @@ -274,109 +310,161 @@ class CheckpointSuite extends TestSuiteBase { testCheckpointedOperation(input, operation, output, 7) } - // This tests whether file input stream remembers what files were seen before // the master failure and uses them again to process a large window operation. // It also tests whether batches, whose processing was incomplete due to the // failure, are re-processed or not. test("recovery with file input stream") { // Set up the streaming context and input streams + val batchDuration = Seconds(2) // Due to 1-second resolution of setLastModified() on some OS's. val testDir = Utils.createTempDir() - var ssc = new StreamingContext(master, framework, Seconds(1)) - ssc.checkpoint(checkpointDir) - val fileStream = ssc.textFileStream(testDir.toString) - // Making value 3 take large time to process, to ensure that the master - // shuts down in the middle of processing the 3rd batch - val mappedStream = fileStream.map(s => { - val i = s.toInt - if (i == 3) Thread.sleep(2000) - i - }) - - // Reducing over a large window to ensure that recovery from master failure - // requires reprocessing of all the files seen before the failure - val reducedStream = mappedStream.reduceByWindow(_ + _, Seconds(30), Seconds(1)) - val outputBuffer = new ArrayBuffer[Seq[Int]] - var outputStream = new TestOutputStream(reducedStream, outputBuffer) - outputStream.register() - ssc.start() - - // Create files and advance manual clock to process them - // var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - Thread.sleep(1000) - for (i <- Seq(1, 2, 3)) { - Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) - // wait to make sure that the file is written such that it gets shown in the file listings - Thread.sleep(1000) + val outputBuffer = new ArrayBuffer[Seq[Int]] with SynchronizedBuffer[Seq[Int]] + + /** + * Writes a file named `i` (which contains the number `i`) to the test directory and sets its + * modification time to `clock`'s current time. + */ + def writeFile(i: Int, clock: ManualClock): Unit = { + val file = new File(testDir, i.toString) + Files.write(i + "\n", file, Charsets.UTF_8) + assert(file.setLastModified(clock.currentTime())) + // Check that the file's modification date is actually the value we wrote, since rounding or + // truncation will break the test: + assert(file.lastModified() === clock.currentTime()) } - logInfo("Output = " + outputStream.output.mkString(",")) - assert(outputStream.output.size > 0, "No files processed before restart") - ssc.stop() - // Verify whether files created have been recorded correctly or not - var fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] - def recordedFiles = fileInputDStream.batchTimeToSelectedFiles.values.flatten - assert(!recordedFiles.filter(_.endsWith("1")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("2")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("3")).isEmpty) - - // Create files while the master is down - for (i <- Seq(4, 5, 6)) { - Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) - Thread.sleep(1000) + /** + * Returns ids that identify which files which have been recorded by the file input stream. + */ + def recordedFiles(ssc: StreamingContext): Seq[Int] = { + val fileInputDStream = + ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] + val filenames = fileInputDStream.batchTimeToSelectedFiles.values.flatten + filenames.map(_.split(File.separator).last.toInt).toSeq.sorted } - // Recover context from checkpoint file and verify whether the files that were - // recorded before failure were saved and successfully recovered - logInfo("*********** RESTARTING ************") - ssc = new StreamingContext(checkpointDir) - fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] - assert(!recordedFiles.filter(_.endsWith("1")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("2")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("3")).isEmpty) + try { + // This is a var because it's re-assigned when we restart from a checkpoint + var clock: ManualClock = null + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + ssc.checkpoint(checkpointDir) + clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val batchCounter = new BatchCounter(ssc) + val fileStream = ssc.textFileStream(testDir.toString) + // Make value 3 take a large time to process, to ensure that the driver + // shuts down in the middle of processing the 3rd batch + CheckpointSuite.batchThreeShouldBlockIndefinitely = true + val mappedStream = fileStream.map(s => { + val i = s.toInt + if (i == 3) { + while (CheckpointSuite.batchThreeShouldBlockIndefinitely) { + Thread.sleep(Long.MaxValue) + } + } + i + }) + + // Reducing over a large window to ensure that recovery from driver failure + // requires reprocessing of all the files seen before the failure + val reducedStream = mappedStream.reduceByWindow(_ + _, batchDuration * 30, batchDuration) + val outputStream = new TestOutputStream(reducedStream, outputBuffer) + outputStream.register() + ssc.start() + + // Advance half a batch so that the first file is created after the StreamingContext starts + clock.addToTime(batchDuration.milliseconds / 2) + // Create files and advance manual clock to process them + for (i <- Seq(1, 2, 3)) { + writeFile(i, clock) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.addToTime(batchDuration.milliseconds) + if (i != 3) { + // Since we want to shut down while the 3rd batch is processing + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === i) + } + } + } + clock.addToTime(batchDuration.milliseconds) + eventually(eventuallyTimeout) { + // Wait until all files have been recorded and all batches have started + assert(recordedFiles(ssc) === Seq(1, 2, 3) && batchCounter.getNumStartedBatches === 3) + } + // Wait for a checkpoint to be written + val fs = new Path(checkpointDir).getFileSystem(ssc.sc.hadoopConfiguration) + eventually(eventuallyTimeout) { + assert(Checkpoint.getCheckpointFiles(checkpointDir, fs).size === 6) + } + ssc.stop() + // Check that we shut down while the third batch was being processed + assert(batchCounter.getNumCompletedBatches === 2) + assert(outputStream.output.flatten === Seq(1, 3)) + } - // Restart stream computation - ssc.start() - for (i <- Seq(7, 8, 9)) { - Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) - Thread.sleep(1000) - } - Thread.sleep(1000) - logInfo("Output = " + outputStream.output.mkString("[", ", ", "]")) - assert(outputStream.output.size > 0, "No files processed after restart") - ssc.stop() + // The original StreamingContext has now been stopped. + CheckpointSuite.batchThreeShouldBlockIndefinitely = false - // Verify whether files created while the driver was down have been recorded or not - assert(!recordedFiles.filter(_.endsWith("4")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("5")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("6")).isEmpty) - - // Verify whether new files created after recover have been recorded or not - assert(!recordedFiles.filter(_.endsWith("7")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("8")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("9")).isEmpty) - - // Append the new output to the old buffer - outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]] - outputBuffer ++= outputStream.output - - val expectedOutput = Seq(1, 3, 6, 10, 15, 21, 28, 36, 45) - logInfo("--------------------------------") - logInfo("output, size = " + outputBuffer.size) - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output, size = " + expectedOutput.size) - expectedOutput.foreach(x => logInfo("[" + x + "]")) - logInfo("--------------------------------") - - // Verify whether all the elements received are as expected - val output = outputBuffer.flatMap(x => x) - assert(output.contains(6)) // To ensure that the 3rd input (i.e., 3) was processed - output.foreach(o => // To ensure all the inputs are correctly added cumulatively - assert(expectedOutput.contains(o), "Expected value " + o + " not found") - ) - // To ensure that all the inputs were received correctly - assert(expectedOutput.last === output.last) - Utils.deleteRecursively(testDir) + // Create files while the streaming driver is down + for (i <- Seq(4, 5, 6)) { + writeFile(i, clock) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.addToTime(batchDuration.milliseconds) + } + + // Recover context from checkpoint file and verify whether the files that were + // recorded before failure were saved and successfully recovered + logInfo("*********** RESTARTING ************") + withStreamingContext(new StreamingContext(checkpointDir)) { ssc => + // So that the restarted StreamingContext's clock has gone forward in time since failure + ssc.conf.set("spark.streaming.manualClock.jump", (batchDuration * 3).milliseconds.toString) + val oldClockTime = clock.currentTime() + clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val batchCounter = new BatchCounter(ssc) + val outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]] + // Check that we remember files that were recorded before the restart + assert(recordedFiles(ssc) === Seq(1, 2, 3)) + + // Restart stream computation + ssc.start() + // Verify that the clock has traveled forward to the expected time + eventually(eventuallyTimeout) { + clock.currentTime() === oldClockTime + } + // Wait for pre-failure batch to be recomputed (3 while SSC was down plus last batch) + val numBatchesAfterRestart = 4 + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === numBatchesAfterRestart) + } + for ((i, index) <- Seq(7, 8, 9).zipWithIndex) { + writeFile(i, clock) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.addToTime(batchDuration.milliseconds) + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === index + numBatchesAfterRestart + 1) + } + } + clock.addToTime(batchDuration.milliseconds) + logInfo("Output after restart = " + outputStream.output.mkString("[", ", ", "]")) + assert(outputStream.output.size > 0, "No files processed after restart") + ssc.stop() + + // Verify whether files created while the driver was down (4, 5, 6) and files created after + // recovery (7, 8, 9) have been recorded + assert(recordedFiles(ssc) === (1 to 9)) + + // Append the new output to the old buffer + outputBuffer ++= outputStream.output + + // Verify whether all the elements received are as expected + val expectedOutput = Seq(1, 3, 6, 10, 15, 21, 28, 36, 45) + assert(outputBuffer.flatten.toSet === expectedOutput.toSet) + } + } finally { + Utils.deleteRecursively(testDir) + } } @@ -433,12 +521,12 @@ class CheckpointSuite extends TestSuiteBase { */ def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - logInfo("Manual clock before advancing = " + clock.time) + logInfo("Manual clock before advancing = " + clock.currentTime()) for (i <- 1 to numBatches.toInt) { clock.addToTime(batchDuration.milliseconds) Thread.sleep(batchDuration.milliseconds) } - logInfo("Manual clock after advancing = " + clock.time) + logInfo("Manual clock after advancing = " + clock.currentTime()) Thread.sleep(batchDuration.milliseconds) val outputStream = ssc.graph.getOutputStreams.filter { dstream => @@ -447,3 +535,7 @@ class CheckpointSuite extends TestSuiteBase { outputStream.output.map(_.flatten) } } + +private object CheckpointSuite extends Serializable { + var batchThreeShouldBlockIndefinitely: Boolean = true +} \ No newline at end of file diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 307052a4a9cbb..bddf51e130422 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -28,7 +28,6 @@ import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer, SynchronizedQueue} -import scala.concurrent.duration._ import scala.language.postfixOps import com.google.common.io.Files @@ -234,45 +233,57 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } def testFileStream(newFilesOnly: Boolean) { - var ssc: StreamingContext = null val testDir: File = null try { + val batchDuration = Seconds(2) val testDir = Utils.createTempDir() + // Create a file that exists before the StreamingContext is created: val existingFile = new File(testDir, "0") Files.write("0\n", existingFile, Charset.forName("UTF-8")) + assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000) - Thread.sleep(1000) // Set up the streaming context and input streams - val newConf = conf.clone.set( - "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") - ssc = new StreamingContext(newConf, batchDuration) - val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( - testDir.toString, (x: Path) => true, newFilesOnly = newFilesOnly).map(_._2.toString) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(fileStream, outputBuffer) - outputStream.register() - ssc.start() - - // Create files in the directory - val input = Seq(1, 2, 3, 4, 5) - input.foreach { i => - Thread.sleep(batchDuration.milliseconds) - val file = new File(testDir, i.toString) - Files.write(i + "\n", file, Charset.forName("UTF-8")) - logInfo("Created file " + file) - } + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + // This `setTime` call ensures that the clock is past the creation time of `existingFile` + clock.setTime(existingFile.lastModified + batchDuration.milliseconds) + val batchCounter = new BatchCounter(ssc) + val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( + testDir.toString, (x: Path) => true, newFilesOnly = newFilesOnly).map(_._2.toString) + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(fileStream, outputBuffer) + outputStream.register() + ssc.start() + + // Advance the clock so that the files are created after StreamingContext starts, but + // not enough to trigger a batch + clock.addToTime(batchDuration.milliseconds / 2) + + // Over time, create files in the directory + val input = Seq(1, 2, 3, 4, 5) + input.foreach { i => + val file = new File(testDir, i.toString) + Files.write(i + "\n", file, Charset.forName("UTF-8")) + assert(file.setLastModified(clock.currentTime())) + assert(file.lastModified === clock.currentTime) + logInfo("Created file " + file) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.addToTime(batchDuration.milliseconds) + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === i) + } + } - // Verify that all the files have been read - val expectedOutput = if (newFilesOnly) { - input.map(_.toString).toSet - } else { - (Seq(0) ++ input).map(_.toString).toSet - } - eventually(timeout(maxWaitTimeMillis milliseconds), interval(100 milliseconds)) { + // Verify that all the files have been read + val expectedOutput = if (newFilesOnly) { + input.map(_.toString).toSet + } else { + (Seq(0) ++ input).map(_.toString).toSet + } assert(outputBuffer.flatten.toSet === expectedOutput) } } finally { - if (ssc != null) ssc.stop() if (testDir != null) Utils.deleteRecursively(testDir) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index 5dbb7232009eb..e0f14fd954280 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming import org.apache.spark.Logging import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils -import org.apache.spark.streaming.StreamingContext._ import scala.util.Random import scala.collection.mutable.ArrayBuffer diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 3661e16a9ef2f..132ff2443fc0f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -168,7 +168,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche manualClock.currentTime() shouldEqual 5000L val cleanupThreshTime = 3000L - handler.cleanupOldBlock(cleanupThreshTime) + handler.cleanupOldBlocks(cleanupThreshTime) eventually(timeout(10000 millis), interval(10 millis)) { getWriteAheadLogFiles().size should be < preCleanupLogFiles.size } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 01a09b67b99dc..de7e9d624bf6b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -166,7 +166,7 @@ class ReceivedBlockTrackerSuite // Cleanup first batch but not second batch val oldestLogFile = getWriteAheadLogFiles().head incrementTime() - tracker3.cleanupOldBatches(batchTime2) + tracker3.cleanupOldBatches(batchTime2, waitForCompletion = true) // Verify that the batch allocations have been cleaned, and the act has been written to log tracker3.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual Seq.empty diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 52972f63c6c5c..7d82c3e4aadcf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -21,11 +21,16 @@ import java.io.{ObjectInputStream, IOException} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.SynchronizedBuffer +import scala.language.implicitConversions import scala.reflect.ClassTag import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.time.{Span, Seconds => ScalaTestSeconds} +import org.scalatest.concurrent.Eventually.timeout +import org.scalatest.concurrent.PatienceConfiguration import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} +import org.apache.spark.streaming.scheduler.{StreamingListenerBatchStarted, StreamingListenerBatchCompleted, StreamingListener} import org.apache.spark.streaming.util.ManualClock import org.apache.spark.{SparkConf, Logging} import org.apache.spark.rdd.RDD @@ -103,6 +108,40 @@ class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T], def toTestOutputStream = new TestOutputStream[T](this.parent, this.output.map(_.flatten)) } +/** + * An object that counts the number of started / completed batches. This is implemented using a + * StreamingListener. Constructing a new instance automatically registers a StreamingListener on + * the given StreamingContext. + */ +class BatchCounter(ssc: StreamingContext) { + + // All access to this state should be guarded by `BatchCounter.this.synchronized` + private var numCompletedBatches = 0 + private var numStartedBatches = 0 + + private val listener = new StreamingListener { + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = + BatchCounter.this.synchronized { + numStartedBatches += 1 + BatchCounter.this.notifyAll() + } + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = + BatchCounter.this.synchronized { + numCompletedBatches += 1 + BatchCounter.this.notifyAll() + } + } + ssc.addStreamingListener(listener) + + def getNumCompletedBatches: Int = this.synchronized { + numCompletedBatches + } + + def getNumStartedBatches: Int = this.synchronized { + numStartedBatches + } +} + /** * This is the base trait for Spark Streaming testsuites. This provides basic functionality * to run user-defined set of input on user-defined stream operations, and verify the output. @@ -142,6 +181,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { .setMaster(master) .setAppName(framework) + // Timeout for use in ScalaTest `eventually` blocks + val eventuallyTimeout: PatienceConfiguration.Timeout = timeout(Span(10, ScalaTestSeconds)) + // Default before function for any streaming test suite. Override this // if you want to add your stuff to "before" (i.e., don't call before { } ) def beforeFunction() { @@ -291,7 +333,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Advance manual clock val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - logInfo("Manual clock before advancing = " + clock.time) + logInfo("Manual clock before advancing = " + clock.currentTime()) if (actuallyWait) { for (i <- 1 to numBatches) { logInfo("Actually waiting for " + batchDuration) @@ -301,7 +343,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { } else { clock.addToTime(numBatches * batchDuration.milliseconds) } - logInfo("Manual clock after advancing = " + clock.time) + logInfo("Manual clock after advancing = " + clock.currentTime()) // Wait until expected number of output items have been generated val startTime = System.currentTimeMillis() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index 471c99fab4682..a5d2bb2fde16c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream import org.apache.spark.storage.StorageLevel diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala index d2b983c4b4d1a..7a6a2f3e577dd 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala @@ -20,15 +20,15 @@ import java.io.File import scala.util.Random -import com.google.common.io.Files import org.apache.hadoop.conf.Configuration -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite} import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter} +import org.apache.spark.util.Utils -class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { +class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { val conf = new SparkConf() .setMaster("local[2]") .setAppName(this.getClass.getSimpleName) @@ -38,16 +38,22 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { var blockManager: BlockManager = null var dir: File = null + override def beforeEach(): Unit = { + dir = Utils.createTempDir() + } + + override def afterEach(): Unit = { + Utils.deleteRecursively(dir) + } + override def beforeAll(): Unit = { sparkContext = new SparkContext(conf) blockManager = sparkContext.env.blockManager - dir = Files.createTempDir() } override def afterAll(): Unit = { // Copied from LocalSparkContext, simpler than to introduced test dependencies to core tests. sparkContext.stop() - dir.delete() System.clearProperty("spark.driver.port") } @@ -137,7 +143,7 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { blockIds: Seq[BlockId] ): Seq[WriteAheadLogFileSegment] = { require(blockData.size === blockIds.size) - val writer = new WriteAheadLogWriter(new File(dir, Random.nextString(10)).toString, hadoopConf) + val writer = new WriteAheadLogWriter(new File(dir, "logFile").toString, hadoopConf) val segments = blockData.zip(blockIds).map { case (data, id) => writer.write(blockManager.dataSerialize(id, data.iterator)) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 1956a4f1db90a..7ce9499dc614d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -22,11 +22,8 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} -import scala.util.Random import WriteAheadLogSuite._ -import com.google.common.io.Files -import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.util.Utils @@ -42,9 +39,9 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { var manager: WriteAheadLogManager = null before { - tempDir = Files.createTempDir() + tempDir = Utils.createTempDir() testDir = tempDir.toString - testFile = new File(tempDir, Random.nextString(10)).toString + testFile = new File(tempDir, "testFile").toString if (manager != null) { manager.stop() manager = null @@ -52,7 +49,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { } after { - FileUtils.deleteQuietly(tempDir) + Utils.deleteRecursively(tempDir) } test("WriteAheadLogWriter - writing data") { @@ -185,15 +182,29 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { } test("WriteAheadLogManager - cleanup old logs") { + logCleanUpTest(waitForCompletion = false) + } + + test("WriteAheadLogManager - cleanup old logs synchronously") { + logCleanUpTest(waitForCompletion = true) + } + + private def logCleanUpTest(waitForCompletion: Boolean): Unit = { // Write data with manager, recover with new manager and verify val manualClock = new ManualClock val dataToWrite = generateRandomData() manager = writeDataUsingManager(testDir, dataToWrite, manualClock, stopManager = false) val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) - manager.cleanupOldLogs(manualClock.currentTime() / 2) - eventually(timeout(1 second), interval(10 milliseconds)) { + + manager.cleanupOldLogs(manualClock.currentTime() / 2, waitForCompletion) + + if (waitForCompletion) { assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } else { + eventually(timeout(1 second), interval(10 milliseconds)) { + assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } } } diff --git a/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala b/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala new file mode 100644 index 0000000000000..d0bf328f2b74d --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala @@ -0,0 +1,35 @@ +/* + * 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.streamingtest + +/** + * A test suite to make sure all `implicit` functions work correctly. + * + * As `implicit` is a compiler feature, we don't need to run this class. + * What we need to do is making the compiler happy. + */ +class ImplicitSuite { + + // We only want to test if `implict` works well with the compiler, so we don't need a real DStream. + def mockDStream[T]: org.apache.spark.streaming.dstream.DStream[T] = null + + def testToPairDStreamFunctions(): Unit = { + val dstream: org.apache.spark.streaming.dstream.DStream[(Int, Int)] = mockDStream + dstream.groupByKey() + } +} diff --git a/tools/pom.xml b/tools/pom.xml index c0bc6e2a2af9d..e7419ed2c607a 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -52,11 +52,6 @@ org.scala-lang scala-compiler
    - - org.scalatest - scalatest_${scala.binary.version} - test - @@ -85,10 +80,6 @@ org.codehaus.mojo build-helper-maven-plugin - - org.scalatest - scalatest-maven-plugin - diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index db58eb642b56d..15ee95070a3d3 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -21,7 +21,7 @@ import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.util.Utils @@ -49,13 +49,13 @@ object StoragePerfTester { val writeData = "1" * recordLength val executor = Executors.newFixedThreadPool(numMaps) - System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.shuffle.sync", "true") - System.setProperty("spark.shuffle.manager", - "org.apache.spark.shuffle.hash.HashShuffleManager") + val conf = new SparkConf() + .set("spark.shuffle.compress", "false") + .set("spark.shuffle.sync", "true") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager") // This is only used to instantiate a BlockManager. All thread scheduling is done manually. - val sc = new SparkContext("local[4]", "Write Tester") + val sc = new SparkContext("local[4]", "Write Tester", conf) val hashShuffleManager = sc.env.shuffleManager.asInstanceOf[HashShuffleManager] def writeOutputBytes(mapId: Int, total: AtomicLong) = { diff --git a/yarn/pom.xml b/yarn/pom.xml index d7579bf9622d6..b86857db7bde6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -64,11 +64,6 @@ tests test - - org.scalatest - scalatest_${scala.binary.version} - test - org.mockito mockito-all @@ -136,22 +131,6 @@ true - - org.apache.maven.plugins - maven-install-plugin - - true - - - - org.scalatest - scalatest-maven-plugin - - - ${basedir}/../.. - - - target/scala-${scala.binary.version}/classes diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 987b3373fb8ff..902bdda59860e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -60,7 +60,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, @volatile private var exitCode = 0 @volatile private var unregistered = false @volatile private var finished = false - @volatile private var finalStatus = FinalApplicationStatus.SUCCEEDED + @volatile private var finalStatus = getDefaultFinalStatus @volatile private var finalMsg: String = "" @volatile private var userClassThread: Thread = _ @@ -102,7 +102,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, logInfo("Invoking sc stop from shutdown hook") sc.stop() } - val maxAppAttempts = client.getMaxRegAttempts(yarnConf) + val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf) val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts if (!finished) { @@ -152,6 +152,20 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, exitCode } + /** + * Set the default final application status for client mode to UNDEFINED to handle + * if YARN HA restarts the application so that it properly retries. Set the final + * status to SUCCEEDED in cluster mode to handle if the user calls System.exit + * from the application code. + */ + final def getDefaultFinalStatus() = { + if (isDriver) { + FinalApplicationStatus.SUCCEEDED + } else { + FinalApplicationStatus.UNDEFINED + } + } + /** * unregister is used to completely unregister the application from the ResourceManager. * This means the ResourceManager will not retry the application attempt on your behalf if @@ -311,7 +325,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def cleanupStagingDir(fs: FileSystem) { var stagingDirPath: Path = null try { - val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = sparkConf.getBoolean("spark.yarn.preserve.staging.files", false) if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { @@ -329,43 +343,43 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def waitForSparkContextInitialized(): SparkContext = { logInfo("Waiting for spark context initialization") - try { - sparkContextRef.synchronized { - var count = 0 - val waitTime = 10000L - val numTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10) - while (sparkContextRef.get() == null && count < numTries && !finished) { - logInfo("Waiting for spark context initialization ... " + count) - count = count + 1 - sparkContextRef.wait(waitTime) - } + sparkContextRef.synchronized { + val waitTries = sparkConf.getOption("spark.yarn.applicationMaster.waitTries") + .map(_.toLong * 10000L) + if (waitTries.isDefined) { + logWarning( + "spark.yarn.applicationMaster.waitTries is deprecated, use spark.yarn.am.waitTime") + } + val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", waitTries.getOrElse(100000L)) + val deadline = System.currentTimeMillis() + totalWaitTime - val sparkContext = sparkContextRef.get() - if (sparkContext == null) { - logError(("SparkContext did not initialize after waiting for %d ms. Please check earlier" - + " log output for errors. Failing the application.").format(numTries * waitTime)) - } - sparkContext + while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { + logInfo("Waiting for spark context initialization ... ") + sparkContextRef.wait(10000L) + } + + val sparkContext = sparkContextRef.get() + if (sparkContext == null) { + logError(("SparkContext did not initialize after waiting for %d ms. Please check earlier" + + " log output for errors. Failing the application.").format(totalWaitTime)) } + sparkContext } } private def waitForSparkDriver(): ActorRef = { logInfo("Waiting for Spark driver to be reachable.") var driverUp = false - var count = 0 val hostport = args.userArgs(0) val (driverHost, driverPort) = Utils.parseHostPort(hostport) - // spark driver should already be up since it launched us, but we don't want to + // Spark driver should already be up since it launched us, but we don't want to // wait forever, so wait 100 seconds max to match the cluster mode setting. - // Leave this config unpublished for now. SPARK-3779 to investigating changing - // this config to be time based. - val numTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 1000) + val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", 100000L) + val deadline = System.currentTimeMillis + totalWaitTime - while (!driverUp && !finished && count < numTries) { + while (!driverUp && !finished && System.currentTimeMillis < deadline) { try { - count = count + 1 val socket = new Socket(driverHost, driverPort) socket.close() logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) @@ -374,7 +388,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, case e: Exception => logError("Failed to connect to driver at %s:%s, retrying ...". format(driverHost, driverPort)) - Thread.sleep(100) + Thread.sleep(100L) } } @@ -511,7 +525,7 @@ object ApplicationMaster extends Logging { SignalLogger.register(log) val amArgs = new ApplicationMasterArguments(args) SparkHadoopUtil.get.runAsSparkUser { () => - master = new ApplicationMaster(amArgs, new YarnRMClientImpl(amArgs)) + master = new ApplicationMaster(amArgs, new YarnRMClient(amArgs)) System.exit(master.run()) } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index addaddb711d3c..c363d755c1752 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -17,34 +17,56 @@ package org.apache.spark.deploy.yarn +import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer -import org.apache.hadoop.conf.Configuration +import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, ListBuffer, Map} +import scala.util.{Try, Success, Failure} + +import com.google.common.base.Objects + import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.permission.FsPermission +import org.apache.hadoop.mapred.Master +import org.apache.hadoop.mapreduce.MRJobConfig +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.util.StringUtils +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.util.Utils -/** - * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's stable API. - */ private[spark] class Client( val args: ClientArguments, val hadoopConf: Configuration, val sparkConf: SparkConf) - extends ClientBase with Logging { + extends Logging { + + import Client._ def this(clientArgs: ClientArguments, spConf: SparkConf) = this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf()) - val yarnClient = YarnClient.createYarnClient - val yarnConf = new YarnConfiguration(hadoopConf) + private val yarnClient = YarnClient.createYarnClient + private val yarnConf = new YarnConfiguration(hadoopConf) + private val credentials = UserGroupInformation.getCurrentUser.getCredentials + private val amMemoryOverhead = args.amMemoryOverhead // MB + private val executorMemoryOverhead = args.executorMemoryOverhead // MB + private val distCacheMgr = new ClientDistributedCacheManager() + private val isClusterMode = args.userClass != null + def stop(): Unit = yarnClient.stop() @@ -61,7 +83,7 @@ private[spark] class Client( * creating applications and setting up the application submission context. This was not * available in the alpha API. */ - override def submitApplication(): ApplicationId = { + def submitApplication(): ApplicationId = { yarnClient.init(yarnConf) yarnClient.start() @@ -98,6 +120,11 @@ private[spark] class Client( appContext.setQueue(args.amQueue) appContext.setAMContainerSpec(containerContext) appContext.setApplicationType("SPARK") + sparkConf.getOption("spark.yarn.maxAppAttempts").map(_.toInt) match { + case Some(v) => appContext.setMaxAppAttempts(v) + case None => logDebug("spark.yarn.maxAppAttempts is not set. " + + "Cluster's default value will be used.") + } val capability = Records.newRecord(classOf[Resource]) capability.setMemory(args.amMemory + amMemoryOverhead) appContext.setResource(capability) @@ -105,25 +132,486 @@ private[spark] class Client( } /** Set up security tokens for launching our ApplicationMaster container. */ - override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { + private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { val dob = new DataOutputBuffer credentials.writeTokenStorageToStream(dob) amContainer.setTokens(ByteBuffer.wrap(dob.getData)) } /** Get the application report from the ResourceManager for an application we have submitted. */ - override def getApplicationReport(appId: ApplicationId): ApplicationReport = + def getApplicationReport(appId: ApplicationId): ApplicationReport = yarnClient.getApplicationReport(appId) /** * Return the security token used by this client to communicate with the ApplicationMaster. * If no security is enabled, the token returned by the report is null. */ - override def getClientToken(report: ApplicationReport): String = + private def getClientToken(report: ApplicationReport): String = Option(report.getClientToAMToken).map(_.toString).getOrElse("") + + /** + * Fail fast if we have requested more resources per container than is available in the cluster. + */ + private def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = { + val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() + logInfo("Verifying our application has not requested more than the maximum " + + s"memory capability of the cluster ($maxMem MB per container)") + val executorMem = args.executorMemory + executorMemoryOverhead + if (executorMem > maxMem) { + throw new IllegalArgumentException(s"Required executor memory (${args.executorMemory}" + + s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") + } + val amMem = args.amMemory + amMemoryOverhead + if (amMem > maxMem) { + throw new IllegalArgumentException(s"Required AM memory (${args.amMemory}" + + s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") + } + logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format( + amMem, + amMemoryOverhead)) + + // We could add checks to make sure the entire cluster has enough resources but that involves + // getting all the node reports and computing ourselves. + } + + /** + * Copy the given file to a remote file system (e.g. HDFS) if needed. + * The file is only copied if the source and destination file systems are different. This is used + * for preparing resources for launching the ApplicationMaster container. Exposed for testing. + */ + private[yarn] def copyFileToRemote( + destDir: Path, + srcPath: Path, + replication: Short, + setPerms: Boolean = false): Path = { + val destFs = destDir.getFileSystem(hadoopConf) + val srcFs = srcPath.getFileSystem(hadoopConf) + var destPath = srcPath + if (!compareFs(srcFs, destFs)) { + destPath = new Path(destDir, srcPath.getName()) + logInfo(s"Uploading resource $srcPath -> $destPath") + FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf) + destFs.setReplication(destPath, replication) + if (setPerms) { + destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION)) + } + } else { + logInfo(s"Source and destination file systems are the same. Not copying $srcPath") + } + // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific + // version shows the specific version in the distributed cache configuration + val qualifiedDestPath = destFs.makeQualified(destPath) + val fc = FileContext.getFileContext(qualifiedDestPath.toUri(), hadoopConf) + fc.resolvePath(qualifiedDestPath) + } + + /** + * Upload any resources to the distributed cache if needed. If a resource is intended to be + * consumed locally, set up the appropriate config for downstream code to handle it properly. + * This is used for setting up a container launch context for our ApplicationMaster. + * Exposed for testing. + */ + def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { + logInfo("Preparing resources for our AM container") + // Upload Spark and the application JAR to the remote file system if necessary, + // and add them as local resources to the application master. + val fs = FileSystem.get(hadoopConf) + val dst = new Path(fs.getHomeDirectory(), appStagingDir) + val nns = getNameNodesToAccess(sparkConf) + dst + obtainTokensForNamenodes(nns, hadoopConf, credentials) + + val replication = sparkConf.getInt("spark.yarn.submit.file.replication", + fs.getDefaultReplication(dst)).toShort + val localResources = HashMap[String, LocalResource]() + FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) + + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + + val oldLog4jConf = Option(System.getenv("SPARK_LOG4J_CONF")) + if (oldLog4jConf.isDefined) { + logWarning( + "SPARK_LOG4J_CONF detected in the system environment. This variable has been " + + "deprecated. Please refer to the \"Launching Spark on YARN\" documentation " + + "for alternatives.") + } + + /** + * Copy the given main resource to the distributed cache if the scheme is not "local". + * Otherwise, set the corresponding key in our SparkConf to handle it downstream. + * Each resource is represented by a 4-tuple of: + * (1) destination resource name, + * (2) local path to the resource, + * (3) Spark property key to set if the scheme is not local, and + * (4) whether to set permissions for this resource + */ + List( + (SPARK_JAR, sparkJar(sparkConf), CONF_SPARK_JAR, false), + (APP_JAR, args.userJar, CONF_SPARK_USER_JAR, true), + ("log4j.properties", oldLog4jConf.orNull, null, false) + ).foreach { case (destName, _localPath, confKey, setPermissions) => + val localPath: String = if (_localPath != null) _localPath.trim() else "" + if (!localPath.isEmpty()) { + val localURI = new URI(localPath) + if (localURI.getScheme != LOCAL_SCHEME) { + val src = getQualifiedLocalPath(localURI, hadoopConf) + val destPath = copyFileToRemote(dst, src, replication, setPermissions) + val destFs = FileSystem.get(destPath.toUri(), hadoopConf) + distCacheMgr.addResource(destFs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, destName, statCache) + } else if (confKey != null) { + // If the resource is intended for local use only, handle this downstream + // by setting the appropriate property + sparkConf.set(confKey, localPath) + } + } + } + + /** + * Do the same for any additional resources passed in through ClientArguments. + * Each resource category is represented by a 3-tuple of: + * (1) comma separated list of resources in this category, + * (2) resource type, and + * (3) whether to add these resources to the classpath + */ + val cachedSecondaryJarLinks = ListBuffer.empty[String] + List( + (args.addJars, LocalResourceType.FILE, true), + (args.files, LocalResourceType.FILE, false), + (args.archives, LocalResourceType.ARCHIVE, false) + ).foreach { case (flist, resType, addToClasspath) => + if (flist != null && !flist.isEmpty()) { + flist.split(',').foreach { file => + val localURI = new URI(file.trim()) + if (localURI.getScheme != LOCAL_SCHEME) { + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyFileToRemote(dst, localPath, replication) + distCacheMgr.addResource( + fs, hadoopConf, destPath, localResources, resType, linkname, statCache) + if (addToClasspath) { + cachedSecondaryJarLinks += linkname + } + } else if (addToClasspath) { + // Resource is intended for local use only and should be added to the class path + cachedSecondaryJarLinks += file.trim() + } + } + } + } + if (cachedSecondaryJarLinks.nonEmpty) { + sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) + } + + localResources + } + + /** + * Set up the environment for launching our ApplicationMaster container. + */ + private def setupLaunchEnv(stagingDir: String): HashMap[String, String] = { + logInfo("Setting up the launch environment for our AM container") + val env = new HashMap[String, String]() + val extraCp = sparkConf.getOption("spark.driver.extraClassPath") + populateClasspath(args, yarnConf, sparkConf, env, extraCp) + env("SPARK_YARN_MODE") = "true" + env("SPARK_YARN_STAGING_DIR") = stagingDir + env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() + + // Set the environment variables to be passed on to the executors. + distCacheMgr.setDistFilesEnv(env) + distCacheMgr.setDistArchivesEnv(env) + + // Pick up any environment variables for the AM provided through spark.yarn.appMasterEnv.* + val amEnvPrefix = "spark.yarn.appMasterEnv." + sparkConf.getAll + .filter { case (k, v) => k.startsWith(amEnvPrefix) } + .map { case (k, v) => (k.substring(amEnvPrefix.length), v) } + .foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) } + + // Keep this for backwards compatibility but users should move to the config + sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + // Allow users to specify some environment variables. + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) + // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments. + env("SPARK_YARN_USER_ENV") = userEnvs + } + + // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to + // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's + // SparkContext will not let that set spark* system properties, which is expected behavior for + // Yarn clients. So propagate it through the environment. + // + // Note that to warn the user about the deprecation in cluster mode, some code from + // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition + // described above). + if (isClusterMode) { + sys.env.get("SPARK_JAVA_OPTS").foreach { value => + val warning = + s""" + |SPARK_JAVA_OPTS was detected (set to '$value'). + |This is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application + | - ./spark-submit with --driver-java-options to set -X options for a driver + | - spark.executor.extraJavaOptions to set -X options for executors + """.stripMargin + logWarning(warning) + for (proc <- Seq("driver", "executor")) { + val key = s"spark.$proc.extraJavaOptions" + if (sparkConf.contains(key)) { + throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.") + } + } + env("SPARK_JAVA_OPTS") = value + } + } + + sys.env.get(ENV_DIST_CLASSPATH).foreach { dcp => + env(ENV_DIST_CLASSPATH) = dcp + } + + env + } + + /** + * Set up a ContainerLaunchContext to launch our ApplicationMaster container. + * This sets up the launch environment, java options, and the command for launching the AM. + */ + private def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) + : ContainerLaunchContext = { + logInfo("Setting up container launch context for our AM") + + val appId = newAppResponse.getApplicationId + val appStagingDir = getAppStagingDir(appId) + val localResources = prepareLocalResources(appStagingDir) + val launchEnv = setupLaunchEnv(appStagingDir) + val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) + amContainer.setLocalResources(localResources) + amContainer.setEnvironment(launchEnv) + + val javaOpts = ListBuffer[String]() + + // Set the environment variable through a command prefix + // to append to the existing value of the variable + var prefixEnv: Option[String] = None + + // Add Xmx for AM memory + javaOpts += "-Xmx" + args.amMemory + "m" + + val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + javaOpts += "-Djava.io.tmpdir=" + tmpDir + + // TODO: Remove once cpuset version is pushed out. + // The context is, default gc for server class machines ends up using all cores to do gc - + // hence if there are multiple containers in same node, Spark GC affects all other containers' + // performance (which can be that of other Spark containers) + // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in + // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset + // of cores on a node. + val useConcurrentAndIncrementalGC = launchEnv.get("SPARK_USE_CONC_INCR_GC").exists(_.toBoolean) + if (useConcurrentAndIncrementalGC) { + // In our expts, using (default) throughput collector has severe perf ramifications in + // multi-tenant machines + javaOpts += "-XX:+UseConcMarkSweepGC" + javaOpts += "-XX:+CMSIncrementalMode" + javaOpts += "-XX:+CMSIncrementalPacing" + javaOpts += "-XX:CMSIncrementalDutyCycleMin=0" + javaOpts += "-XX:CMSIncrementalDutyCycle=10" + } + + // Forward the Spark configuration to the application master / executors. + // TODO: it might be nicer to pass these as an internal environment variable rather than + // as Java options, due to complications with string parsing of nested quotes. + for ((k, v) <- sparkConf.getAll) { + javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") + } + + // Include driver-specific java options if we are launching a driver + if (isClusterMode) { + sparkConf.getOption("spark.driver.extraJavaOptions") + .orElse(sys.env.get("SPARK_JAVA_OPTS")) + .map(Utils.splitCommandString).getOrElse(Seq.empty) + .foreach(opts => javaOpts += opts) + val libraryPaths = Seq(sys.props.get("spark.driver.extraLibraryPath"), + sys.props.get("spark.driver.libraryPath")).flatten + if (libraryPaths.nonEmpty) { + prefixEnv = Some(Utils.libraryPathEnvPrefix(libraryPaths)) + } + if (sparkConf.getOption("spark.yarn.am.extraJavaOptions").isDefined) { + logWarning("spark.yarn.am.extraJavaOptions will not take effect in cluster mode") + } + } else { + // Validate and include yarn am specific java options in yarn-client mode. + val amOptsKey = "spark.yarn.am.extraJavaOptions" + val amOpts = sparkConf.getOption(amOptsKey) + amOpts.foreach { opts => + if (opts.contains("-Dspark")) { + val msg = s"$amOptsKey is not allowed to set Spark options (was '$opts'). " + throw new SparkException(msg) + } + if (opts.contains("-Xmx") || opts.contains("-Xms")) { + val msg = s"$amOptsKey is not allowed to alter memory settings (was '$opts')." + throw new SparkException(msg) + } + javaOpts ++= Utils.splitCommandString(opts) + } + } + + // For log4j configuration to reference + javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) + + val userClass = + if (isClusterMode) { + Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass)) + } else { + Nil + } + val userJar = + if (args.userJar != null) { + Seq("--jar", args.userJar) + } else { + Nil + } + val amClass = + if (isClusterMode) { + Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName + } else { + Class.forName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName + } + val userArgs = args.userArgs.flatMap { arg => + Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) + } + val amArgs = + Seq(amClass) ++ userClass ++ userJar ++ userArgs ++ + Seq( + "--executor-memory", args.executorMemory.toString + "m", + "--executor-cores", args.executorCores.toString, + "--num-executors ", args.numExecutors.toString) + + // Command for the ApplicationMaster + val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ + javaOpts ++ amArgs ++ + Seq( + "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", + "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + // TODO: it would be nicer to just make sure there are no null commands here + val printableCommands = commands.map(s => if (s == null) "null" else s).toList + amContainer.setCommands(printableCommands) + + logDebug("===============================================================================") + logDebug("Yarn AM launch context:") + logDebug(s" user class: ${Option(args.userClass).getOrElse("N/A")}") + logDebug(" env:") + launchEnv.foreach { case (k, v) => logDebug(s" $k -> $v") } + logDebug(" resources:") + localResources.foreach { case (k, v) => logDebug(s" $k -> $v")} + logDebug(" command:") + logDebug(s" ${printableCommands.mkString(" ")}") + logDebug("===============================================================================") + + // send the acl settings into YARN to control who has access via YARN interfaces + val securityManager = new SecurityManager(sparkConf) + amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager)) + setupSecurityToken(amContainer) + UserGroupInformation.getCurrentUser().addCredentials(credentials) + + amContainer + } + + /** + * Report the state of an application until it has exited, either successfully or + * due to some failure, then return a pair of the yarn application state (FINISHED, FAILED, + * KILLED, or RUNNING) and the final application state (UNDEFINED, SUCCEEDED, FAILED, + * or KILLED). + * + * @param appId ID of the application to monitor. + * @param returnOnRunning Whether to also return the application state when it is RUNNING. + * @param logApplicationReport Whether to log details of the application report every iteration. + * @return A pair of the yarn application state and the final application state. + */ + def monitorApplication( + appId: ApplicationId, + returnOnRunning: Boolean = false, + logApplicationReport: Boolean = true): (YarnApplicationState, FinalApplicationStatus) = { + val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) + var lastState: YarnApplicationState = null + while (true) { + Thread.sleep(interval) + val report = getApplicationReport(appId) + val state = report.getYarnApplicationState + + if (logApplicationReport) { + logInfo(s"Application report for $appId (state: $state)") + val details = Seq[(String, String)]( + ("client token", getClientToken(report)), + ("diagnostics", report.getDiagnostics), + ("ApplicationMaster host", report.getHost), + ("ApplicationMaster RPC port", report.getRpcPort.toString), + ("queue", report.getQueue), + ("start time", report.getStartTime.toString), + ("final status", report.getFinalApplicationStatus.toString), + ("tracking URL", report.getTrackingUrl), + ("user", report.getUser) + ) + + // Use more loggable format if value is null or empty + val formattedDetails = details + .map { case (k, v) => + val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") + s"\n\t $k: $newValue" } + .mkString("") + + // If DEBUG is enabled, log report details every iteration + // Otherwise, log them every time the application changes state + if (log.isDebugEnabled) { + logDebug(formattedDetails) + } else if (lastState != state) { + logInfo(formattedDetails) + } + } + + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + return (state, report.getFinalApplicationStatus) + } + + if (returnOnRunning && state == YarnApplicationState.RUNNING) { + return (state, report.getFinalApplicationStatus) + } + + lastState = state + } + + // Never reached, but keeps compiler happy + throw new SparkException("While loop is depleted! This should never happen...") + } + + /** + * Submit an application to the ResourceManager and monitor its state. + * This continues until the application has exited for any reason. + * If the application finishes with a failed, killed, or undefined status, + * throw an appropriate SparkException. + */ + def run(): Unit = { + val (yarnApplicationState, finalApplicationStatus) = monitorApplication(submitApplication()) + if (yarnApplicationState == YarnApplicationState.FAILED || + finalApplicationStatus == FinalApplicationStatus.FAILED) { + throw new SparkException("Application finished with failed status") + } + if (yarnApplicationState == YarnApplicationState.KILLED || + finalApplicationStatus == FinalApplicationStatus.KILLED) { + throw new SparkException("Application is killed") + } + if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) { + throw new SparkException("The final status of application is undefined") + } + } } -object Client { +object Client extends Logging { def main(argStrings: Array[String]) { if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a " + @@ -138,4 +626,311 @@ object Client { val args = new ClientArguments(argStrings, sparkConf) new Client(args, sparkConf).run() } + + // Alias for the Spark assembly jar and the user jar + val SPARK_JAR: String = "__spark__.jar" + val APP_JAR: String = "__app__.jar" + + // URI scheme that identifies local resources + val LOCAL_SCHEME = "local" + + // Staging directory for any temporary jars or files + val SPARK_STAGING: String = ".sparkStaging" + + // Location of any user-defined Spark jars + val CONF_SPARK_JAR = "spark.yarn.jar" + val ENV_SPARK_JAR = "SPARK_JAR" + + // Internal config to propagate the location of the user's jar to the driver/executors + val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" + + // Internal config to propagate the locations of any extra jars to add to the classpath + // of the executors + val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" + + // Staging directory is private! -> rwx-------- + val STAGING_DIR_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) + + // App files are world-wide readable and owner writable -> rw-r--r-- + val APP_FILE_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) + + // Distribution-defined classpath to add to processes + val ENV_DIST_CLASSPATH = "SPARK_DIST_CLASSPATH" + + /** + * Find the user-defined Spark jar if configured, or return the jar containing this + * class if not. + * + * This method first looks in the SparkConf object for the CONF_SPARK_JAR key, and in the + * user environment if that is not found (for backwards compatibility). + */ + private def sparkJar(conf: SparkConf): String = { + if (conf.contains(CONF_SPARK_JAR)) { + conf.get(CONF_SPARK_JAR) + } else if (System.getenv(ENV_SPARK_JAR) != null) { + logWarning( + s"$ENV_SPARK_JAR detected in the system environment. This variable has been deprecated " + + s"in favor of the $CONF_SPARK_JAR configuration variable.") + System.getenv(ENV_SPARK_JAR) + } else { + SparkContext.jarOfClass(this.getClass).head + } + } + + /** + * Return the path to the given application's staging directory. + */ + private def getAppStagingDir(appId: ApplicationId): String = { + SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR + } + + /** + * Populate the classpath entry in the given environment map with any application + * classpath specified through the Hadoop and Yarn configurations. + */ + private[yarn] def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) + : Unit = { + val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf) + for (c <- classPathElementsToAdd.flatten) { + YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, c.trim) + } + } + + private def getYarnAppClasspath(conf: Configuration): Option[Seq[String]] = + Option(conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) match { + case Some(s) => Some(s.toSeq) + case None => getDefaultYarnApplicationClasspath + } + + private def getMRAppClasspath(conf: Configuration): Option[Seq[String]] = + Option(conf.getStrings("mapreduce.application.classpath")) match { + case Some(s) => Some(s.toSeq) + case None => getDefaultMRApplicationClasspath + } + + private[yarn] def getDefaultYarnApplicationClasspath: Option[Seq[String]] = { + val triedDefault = Try[Seq[String]] { + val field = classOf[YarnConfiguration].getField("DEFAULT_YARN_APPLICATION_CLASSPATH") + val value = field.get(null).asInstanceOf[Array[String]] + value.toSeq + } recoverWith { + case e: NoSuchFieldException => Success(Seq.empty[String]) + } + + triedDefault match { + case f: Failure[_] => + logError("Unable to obtain the default YARN Application classpath.", f.exception) + case s: Success[Seq[String]] => + logDebug(s"Using the default YARN application classpath: ${s.get.mkString(",")}") + } + + triedDefault.toOption + } + + /** + * In Hadoop 0.23, the MR application classpath comes with the YARN application + * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. + * So we need to use reflection to retrieve it. + */ + private[yarn] def getDefaultMRApplicationClasspath: Option[Seq[String]] = { + val triedDefault = Try[Seq[String]] { + val field = classOf[MRJobConfig].getField("DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH") + val value = if (field.getType == classOf[String]) { + StringUtils.getStrings(field.get(null).asInstanceOf[String]).toArray + } else { + field.get(null).asInstanceOf[Array[String]] + } + value.toSeq + } recoverWith { + case e: NoSuchFieldException => Success(Seq.empty[String]) + } + + triedDefault match { + case f: Failure[_] => + logError("Unable to obtain the default MR Application classpath.", f.exception) + case s: Success[Seq[String]] => + logDebug(s"Using the default MR application classpath: ${s.get.mkString(",")}") + } + + triedDefault.toOption + } + + /** + * Populate the classpath entry in the given environment map. + * This includes the user jar, Spark jar, and any extra application jars. + */ + private[yarn] def populateClasspath( + args: ClientArguments, + conf: Configuration, + sparkConf: SparkConf, + env: HashMap[String, String], + extraClassPath: Option[String] = None): Unit = { + extraClassPath.foreach(addClasspathEntry(_, env)) + addClasspathEntry(Environment.PWD.$(), env) + + // Normally the users app.jar is last in case conflicts with spark jars + if (sparkConf.getBoolean("spark.yarn.user.classpath.first", false)) { + addUserClasspath(args, sparkConf, env) + addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) + populateHadoopClasspath(conf, env) + } else { + addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) + populateHadoopClasspath(conf, env) + addUserClasspath(args, sparkConf, env) + } + + // Append all jar files under the working directory to the classpath. + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env) + } + + /** + * Adds the user jars which have local: URIs (or alternate names, such as APP_JAR) explicitly + * to the classpath. + */ + private def addUserClasspath( + args: ClientArguments, + conf: SparkConf, + env: HashMap[String, String]): Unit = { + + // If `args` is not null, we are launching an AM container. + // Otherwise, we are launching executor containers. + val (mainJar, secondaryJars) = + if (args != null) { + (args.userJar, args.addJars) + } else { + (conf.get(CONF_SPARK_USER_JAR, null), conf.get(CONF_SPARK_YARN_SECONDARY_JARS, null)) + } + + addFileToClasspath(mainJar, APP_JAR, env) + if (secondaryJars != null) { + secondaryJars.split(",").filter(_.nonEmpty).foreach { jar => + addFileToClasspath(jar, null, env) + } + } + } + + /** + * Adds the given path to the classpath, handling "local:" URIs correctly. + * + * If an alternate name for the file is given, and it's not a "local:" file, the alternate + * name will be added to the classpath (relative to the job's work directory). + * + * If not a "local:" file and no alternate name, the environment is not modified. + * + * @param path Path to add to classpath (optional). + * @param fileName Alternate name for the file (optional). + * @param env Map holding the environment variables. + */ + private def addFileToClasspath( + path: String, + fileName: String, + env: HashMap[String, String]): Unit = { + if (path != null) { + scala.util.control.Exception.ignoring(classOf[URISyntaxException]) { + val uri = new URI(path) + if (uri.getScheme == LOCAL_SCHEME) { + addClasspathEntry(uri.getPath, env) + return + } + } + } + if (fileName != null) { + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env) + } + } + + /** + * Add the given path to the classpath entry of the given environment map. + * If the classpath is already set, this appends the new path to the existing classpath. + */ + private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit = + YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path) + + /** + * Get the list of namenodes the user may access. + */ + private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { + sparkConf.get("spark.yarn.access.namenodes", "") + .split(",") + .map(_.trim()) + .filter(!_.isEmpty) + .map(new Path(_)) + .toSet + } + + private[yarn] def getTokenRenewer(conf: Configuration): String = { + val delegTokenRenewer = Master.getMasterPrincipal(conf) + logDebug("delegation token renewer is: " + delegTokenRenewer) + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer" + logError(errorMessage) + throw new SparkException(errorMessage) + } + delegTokenRenewer + } + + /** + * Obtains tokens for the namenodes passed in and adds them to the credentials. + */ + private def obtainTokensForNamenodes( + paths: Set[Path], + conf: Configuration, + creds: Credentials): Unit = { + if (UserGroupInformation.isSecurityEnabled()) { + val delegTokenRenewer = getTokenRenewer(conf) + paths.foreach { dst => + val dstFs = dst.getFileSystem(conf) + logDebug("getting token for namenode: " + dst) + dstFs.addDelegationTokens(delegTokenRenewer, creds) + } + } + } + + /** + * Return whether the two file systems are the same. + */ + private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { + val srcUri = srcFs.getUri() + val dstUri = destFs.getUri() + if (srcUri.getScheme() == null || srcUri.getScheme() != dstUri.getScheme()) { + return false + } + + var srcHost = srcUri.getHost() + var dstHost = dstUri.getHost() + + // In HA or when using viewfs, the host part of the URI may not actually be a host, but the + // name of the HDFS namespace. Those names won't resolve, so avoid even trying if they + // match. + if (srcHost != null && dstHost != null && srcHost != dstHost) { + try { + srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() + dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() + } catch { + case e: UnknownHostException => + return false + } + } + + Objects.equal(srcHost, dstHost) && srcUri.getPort() == dstUri.getPort() + } + + /** + * Given a local URI, resolve it and return a qualified local path that corresponds to the URI. + * This is used for preparing local resources to be included in the container launch context. + */ + private def getQualifiedLocalPath(localURI: URI, hadoopConf: Configuration): Path = { + val qualifiedURI = + if (localURI.getScheme == null) { + // If not specified, assume this is in the local filesystem to keep the behavior + // consistent with that of Hadoop + new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(localURI)).toString) + } else { + localURI + } + new Path(qualifiedURI) + } + } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index c439969510fc9..39f1021c9d942 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -39,6 +39,8 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var appName: String = "Spark" var priority = 0 + parseArgs(args.toList) + // Additional memory to allocate to containers // For now, use driver's memory overhead as our AM container's memory overhead val amMemoryOverhead = sparkConf.getInt("spark.yarn.driver.memoryOverhead", @@ -50,7 +52,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) private val isDynamicAllocationEnabled = sparkConf.getBoolean("spark.dynamicAllocation.enabled", false) - parseArgs(args.toList) loadEnvironmentArgs() validateArgs() @@ -197,6 +198,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | to work with. | --files files Comma separated list of files to be distributed with the job. | --archives archives Comma separated list of archives to be distributed with the job. - """ + """.stripMargin } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala deleted file mode 100644 index f95d72379171c..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ /dev/null @@ -1,823 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} - -import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, ListBuffer, Map} -import scala.util.{Try, Success, Failure} - -import com.google.common.base.Objects -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs._ -import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.mapred.Master -import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.hadoop.util.StringUtils -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.Records - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} -import org.apache.spark.util.Utils - -/** - * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. - * The Client submits an application to the YARN ResourceManager. - */ -private[spark] trait ClientBase extends Logging { - import ClientBase._ - - protected val args: ClientArguments - protected val hadoopConf: Configuration - protected val sparkConf: SparkConf - protected val yarnConf: YarnConfiguration - protected val credentials = UserGroupInformation.getCurrentUser.getCredentials - protected val amMemoryOverhead = args.amMemoryOverhead // MB - protected val executorMemoryOverhead = args.executorMemoryOverhead // MB - private val distCacheMgr = new ClientDistributedCacheManager() - private val isLaunchingDriver = args.userClass != null - - /** - * Fail fast if we have requested more resources per container than is available in the cluster. - */ - protected def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = { - val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() - logInfo("Verifying our application has not requested more than the maximum " + - s"memory capability of the cluster ($maxMem MB per container)") - val executorMem = args.executorMemory + executorMemoryOverhead - if (executorMem > maxMem) { - throw new IllegalArgumentException(s"Required executor memory (${args.executorMemory}" + - s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") - } - val amMem = args.amMemory + amMemoryOverhead - if (amMem > maxMem) { - throw new IllegalArgumentException(s"Required AM memory (${args.amMemory}" + - s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") - } - logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format( - amMem, - amMemoryOverhead)) - - // We could add checks to make sure the entire cluster has enough resources but that involves - // getting all the node reports and computing ourselves. - } - - /** - * Copy the given file to a remote file system (e.g. HDFS) if needed. - * The file is only copied if the source and destination file systems are different. This is used - * for preparing resources for launching the ApplicationMaster container. Exposed for testing. - */ - def copyFileToRemote( - destDir: Path, - srcPath: Path, - replication: Short, - setPerms: Boolean = false): Path = { - val destFs = destDir.getFileSystem(hadoopConf) - val srcFs = srcPath.getFileSystem(hadoopConf) - var destPath = srcPath - if (!compareFs(srcFs, destFs)) { - destPath = new Path(destDir, srcPath.getName()) - logInfo(s"Uploading resource $srcPath -> $destPath") - FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf) - destFs.setReplication(destPath, replication) - if (setPerms) { - destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION)) - } - } else { - logInfo(s"Source and destination file systems are the same. Not copying $srcPath") - } - // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific - // version shows the specific version in the distributed cache configuration - val qualifiedDestPath = destFs.makeQualified(destPath) - val fc = FileContext.getFileContext(qualifiedDestPath.toUri(), hadoopConf) - fc.resolvePath(qualifiedDestPath) - } - - /** - * Given a local URI, resolve it and return a qualified local path that corresponds to the URI. - * This is used for preparing local resources to be included in the container launch context. - */ - private def getQualifiedLocalPath(localURI: URI): Path = { - val qualifiedURI = - if (localURI.getScheme == null) { - // If not specified, assume this is in the local filesystem to keep the behavior - // consistent with that of Hadoop - new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(localURI)).toString) - } else { - localURI - } - new Path(qualifiedURI) - } - - /** - * Upload any resources to the distributed cache if needed. If a resource is intended to be - * consumed locally, set up the appropriate config for downstream code to handle it properly. - * This is used for setting up a container launch context for our ApplicationMaster. - * Exposed for testing. - */ - def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { - logInfo("Preparing resources for our AM container") - // Upload Spark and the application JAR to the remote file system if necessary, - // and add them as local resources to the application master. - val fs = FileSystem.get(hadoopConf) - val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val nns = getNameNodesToAccess(sparkConf) + dst - obtainTokensForNamenodes(nns, hadoopConf, credentials) - - val replication = sparkConf.getInt("spark.yarn.submit.file.replication", - fs.getDefaultReplication(dst)).toShort - val localResources = HashMap[String, LocalResource]() - FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) - - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - - val oldLog4jConf = Option(System.getenv("SPARK_LOG4J_CONF")) - if (oldLog4jConf.isDefined) { - logWarning( - "SPARK_LOG4J_CONF detected in the system environment. This variable has been " + - "deprecated. Please refer to the \"Launching Spark on YARN\" documentation " + - "for alternatives.") - } - - /** - * Copy the given main resource to the distributed cache if the scheme is not "local". - * Otherwise, set the corresponding key in our SparkConf to handle it downstream. - * Each resource is represented by a 4-tuple of: - * (1) destination resource name, - * (2) local path to the resource, - * (3) Spark property key to set if the scheme is not local, and - * (4) whether to set permissions for this resource - */ - List( - (SPARK_JAR, sparkJar(sparkConf), CONF_SPARK_JAR, false), - (APP_JAR, args.userJar, CONF_SPARK_USER_JAR, true), - ("log4j.properties", oldLog4jConf.orNull, null, false) - ).foreach { case (destName, _localPath, confKey, setPermissions) => - val localPath: String = if (_localPath != null) _localPath.trim() else "" - if (!localPath.isEmpty()) { - val localURI = new URI(localPath) - if (localURI.getScheme != LOCAL_SCHEME) { - val src = getQualifiedLocalPath(localURI) - val destPath = copyFileToRemote(dst, src, replication, setPermissions) - val destFs = FileSystem.get(destPath.toUri(), hadoopConf) - distCacheMgr.addResource(destFs, hadoopConf, destPath, - localResources, LocalResourceType.FILE, destName, statCache) - } else if (confKey != null) { - // If the resource is intended for local use only, handle this downstream - // by setting the appropriate property - sparkConf.set(confKey, localPath) - } - } - } - - /** - * Do the same for any additional resources passed in through ClientArguments. - * Each resource category is represented by a 3-tuple of: - * (1) comma separated list of resources in this category, - * (2) resource type, and - * (3) whether to add these resources to the classpath - */ - val cachedSecondaryJarLinks = ListBuffer.empty[String] - List( - (args.addJars, LocalResourceType.FILE, true), - (args.files, LocalResourceType.FILE, false), - (args.archives, LocalResourceType.ARCHIVE, false) - ).foreach { case (flist, resType, addToClasspath) => - if (flist != null && !flist.isEmpty()) { - flist.split(',').foreach { file => - val localURI = new URI(file.trim()) - if (localURI.getScheme != LOCAL_SCHEME) { - val localPath = new Path(localURI) - val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) - val destPath = copyFileToRemote(dst, localPath, replication) - distCacheMgr.addResource( - fs, hadoopConf, destPath, localResources, resType, linkname, statCache) - if (addToClasspath) { - cachedSecondaryJarLinks += linkname - } - } else if (addToClasspath) { - // Resource is intended for local use only and should be added to the class path - cachedSecondaryJarLinks += file.trim() - } - } - } - } - if (cachedSecondaryJarLinks.nonEmpty) { - sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) - } - - localResources - } - - /** - * Set up the environment for launching our ApplicationMaster container. - */ - private def setupLaunchEnv(stagingDir: String): HashMap[String, String] = { - logInfo("Setting up the launch environment for our AM container") - val env = new HashMap[String, String]() - val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - populateClasspath(args, yarnConf, sparkConf, env, extraCp) - env("SPARK_YARN_MODE") = "true" - env("SPARK_YARN_STAGING_DIR") = stagingDir - env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() - - // Set the environment variables to be passed on to the executors. - distCacheMgr.setDistFilesEnv(env) - distCacheMgr.setDistArchivesEnv(env) - - // Pick up any environment variables for the AM provided through spark.yarn.appMasterEnv.* - val amEnvPrefix = "spark.yarn.appMasterEnv." - sparkConf.getAll - .filter { case (k, v) => k.startsWith(amEnvPrefix) } - .map { case (k, v) => (k.substring(amEnvPrefix.length), v) } - .foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) } - - // Keep this for backwards compatibility but users should move to the config - sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => - // Allow users to specify some environment variables. - YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) - // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments. - env("SPARK_YARN_USER_ENV") = userEnvs - } - - // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to - // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's - // SparkContext will not let that set spark* system properties, which is expected behavior for - // Yarn clients. So propagate it through the environment. - // - // Note that to warn the user about the deprecation in cluster mode, some code from - // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition - // described above). - if (isLaunchingDriver) { - sys.env.get("SPARK_JAVA_OPTS").foreach { value => - val warning = - s""" - |SPARK_JAVA_OPTS was detected (set to '$value'). - |This is deprecated in Spark 1.0+. - | - |Please instead use: - | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application - | - ./spark-submit with --driver-java-options to set -X options for a driver - | - spark.executor.extraJavaOptions to set -X options for executors - """.stripMargin - logWarning(warning) - for (proc <- Seq("driver", "executor")) { - val key = s"spark.$proc.extraJavaOptions" - if (sparkConf.contains(key)) { - throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.") - } - } - env("SPARK_JAVA_OPTS") = value - } - } - - env - } - - /** - * Set up a ContainerLaunchContext to launch our ApplicationMaster container. - * This sets up the launch environment, java options, and the command for launching the AM. - */ - protected def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) - : ContainerLaunchContext = { - logInfo("Setting up container launch context for our AM") - - val appId = newAppResponse.getApplicationId - val appStagingDir = getAppStagingDir(appId) - val localResources = prepareLocalResources(appStagingDir) - val launchEnv = setupLaunchEnv(appStagingDir) - val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) - amContainer.setLocalResources(localResources) - amContainer.setEnvironment(launchEnv) - - val javaOpts = ListBuffer[String]() - - // Set the environment variable through a command prefix - // to append to the existing value of the variable - var prefixEnv: Option[String] = None - - // Add Xmx for AM memory - javaOpts += "-Xmx" + args.amMemory + "m" - - val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) - javaOpts += "-Djava.io.tmpdir=" + tmpDir - - // TODO: Remove once cpuset version is pushed out. - // The context is, default gc for server class machines ends up using all cores to do gc - - // hence if there are multiple containers in same node, Spark GC affects all other containers' - // performance (which can be that of other Spark containers) - // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in - // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset - // of cores on a node. - val useConcurrentAndIncrementalGC = launchEnv.get("SPARK_USE_CONC_INCR_GC").exists(_.toBoolean) - if (useConcurrentAndIncrementalGC) { - // In our expts, using (default) throughput collector has severe perf ramifications in - // multi-tenant machines - javaOpts += "-XX:+UseConcMarkSweepGC" - javaOpts += "-XX:+CMSIncrementalMode" - javaOpts += "-XX:+CMSIncrementalPacing" - javaOpts += "-XX:CMSIncrementalDutyCycleMin=0" - javaOpts += "-XX:CMSIncrementalDutyCycle=10" - } - - // Forward the Spark configuration to the application master / executors. - // TODO: it might be nicer to pass these as an internal environment variable rather than - // as Java options, due to complications with string parsing of nested quotes. - for ((k, v) <- sparkConf.getAll) { - javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") - } - - // Include driver-specific java options if we are launching a driver - if (isLaunchingDriver) { - sparkConf.getOption("spark.driver.extraJavaOptions") - .orElse(sys.env.get("SPARK_JAVA_OPTS")) - .foreach(opts => javaOpts += opts) - val libraryPaths = Seq(sys.props.get("spark.driver.extraLibraryPath"), - sys.props.get("spark.driver.libraryPath")).flatten - if (libraryPaths.nonEmpty) { - prefixEnv = Some(Utils.libraryPathEnvPrefix(libraryPaths)) - } - } - - // For log4j configuration to reference - javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) - - val userClass = - if (isLaunchingDriver) { - Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass)) - } else { - Nil - } - val userJar = - if (args.userJar != null) { - Seq("--jar", args.userJar) - } else { - Nil - } - val amClass = - if (isLaunchingDriver) { - Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName - } else { - Class.forName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName - } - val userArgs = args.userArgs.flatMap { arg => - Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) - } - val amArgs = - Seq(amClass) ++ userClass ++ userJar ++ userArgs ++ - Seq( - "--executor-memory", args.executorMemory.toString + "m", - "--executor-cores", args.executorCores.toString, - "--num-executors ", args.numExecutors.toString) - - // Command for the ApplicationMaster - val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ - javaOpts ++ amArgs ++ - Seq( - "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", - "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - - // TODO: it would be nicer to just make sure there are no null commands here - val printableCommands = commands.map(s => if (s == null) "null" else s).toList - amContainer.setCommands(printableCommands) - - logDebug("===============================================================================") - logDebug("Yarn AM launch context:") - logDebug(s" user class: ${Option(args.userClass).getOrElse("N/A")}") - logDebug(" env:") - launchEnv.foreach { case (k, v) => logDebug(s" $k -> $v") } - logDebug(" resources:") - localResources.foreach { case (k, v) => logDebug(s" $k -> $v")} - logDebug(" command:") - logDebug(s" ${printableCommands.mkString(" ")}") - logDebug("===============================================================================") - - // send the acl settings into YARN to control who has access via YARN interfaces - val securityManager = new SecurityManager(sparkConf) - amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager)) - setupSecurityToken(amContainer) - UserGroupInformation.getCurrentUser().addCredentials(credentials) - - amContainer - } - - /** - * Report the state of an application until it has exited, either successfully or - * due to some failure, then return a pair of the yarn application state (FINISHED, FAILED, - * KILLED, or RUNNING) and the final application state (UNDEFINED, SUCCEEDED, FAILED, - * or KILLED). - * - * @param appId ID of the application to monitor. - * @param returnOnRunning Whether to also return the application state when it is RUNNING. - * @param logApplicationReport Whether to log details of the application report every iteration. - * @return A pair of the yarn application state and the final application state. - */ - def monitorApplication( - appId: ApplicationId, - returnOnRunning: Boolean = false, - logApplicationReport: Boolean = true): (YarnApplicationState, FinalApplicationStatus) = { - val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - var lastState: YarnApplicationState = null - while (true) { - Thread.sleep(interval) - val report = getApplicationReport(appId) - val state = report.getYarnApplicationState - - if (logApplicationReport) { - logInfo(s"Application report for $appId (state: $state)") - val details = Seq[(String, String)]( - ("client token", getClientToken(report)), - ("diagnostics", report.getDiagnostics), - ("ApplicationMaster host", report.getHost), - ("ApplicationMaster RPC port", report.getRpcPort.toString), - ("queue", report.getQueue), - ("start time", report.getStartTime.toString), - ("final status", report.getFinalApplicationStatus.toString), - ("tracking URL", report.getTrackingUrl), - ("user", report.getUser) - ) - - // Use more loggable format if value is null or empty - val formattedDetails = details - .map { case (k, v) => - val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") - s"\n\t $k: $newValue" } - .mkString("") - - // If DEBUG is enabled, log report details every iteration - // Otherwise, log them every time the application changes state - if (log.isDebugEnabled) { - logDebug(formattedDetails) - } else if (lastState != state) { - logInfo(formattedDetails) - } - } - - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - return (state, report.getFinalApplicationStatus) - } - - if (returnOnRunning && state == YarnApplicationState.RUNNING) { - return (state, report.getFinalApplicationStatus) - } - - lastState = state - } - - // Never reached, but keeps compiler happy - throw new SparkException("While loop is depleted! This should never happen...") - } - - /** - * Submit an application to the ResourceManager and monitor its state. - * This continues until the application has exited for any reason. - * If the application finishes with a failed, killed, or undefined status, - * throw an appropriate SparkException. - */ - def run(): Unit = { - val (yarnApplicationState, finalApplicationStatus) = monitorApplication(submitApplication()) - if (yarnApplicationState == YarnApplicationState.FAILED || - finalApplicationStatus == FinalApplicationStatus.FAILED) { - throw new SparkException("Application finished with failed status") - } - if (yarnApplicationState == YarnApplicationState.KILLED || - finalApplicationStatus == FinalApplicationStatus.KILLED) { - throw new SparkException("Application is killed") - } - if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) { - throw new SparkException("The final status of application is undefined") - } - } - - /* --------------------------------------------------------------------------------------- * - | Methods that cannot be implemented here due to API differences across hadoop versions | - * --------------------------------------------------------------------------------------- */ - - /** Submit an application running our ApplicationMaster to the ResourceManager. */ - def submitApplication(): ApplicationId - - /** Set up security tokens for launching our ApplicationMaster container. */ - protected def setupSecurityToken(containerContext: ContainerLaunchContext): Unit - - /** Get the application report from the ResourceManager for an application we have submitted. */ - protected def getApplicationReport(appId: ApplicationId): ApplicationReport - - /** - * Return the security token used by this client to communicate with the ApplicationMaster. - * If no security is enabled, the token returned by the report is null. - */ - protected def getClientToken(report: ApplicationReport): String -} - -private[spark] object ClientBase extends Logging { - - // Alias for the Spark assembly jar and the user jar - val SPARK_JAR: String = "__spark__.jar" - val APP_JAR: String = "__app__.jar" - - // URI scheme that identifies local resources - val LOCAL_SCHEME = "local" - - // Staging directory for any temporary jars or files - val SPARK_STAGING: String = ".sparkStaging" - - // Location of any user-defined Spark jars - val CONF_SPARK_JAR = "spark.yarn.jar" - val ENV_SPARK_JAR = "SPARK_JAR" - - // Internal config to propagate the location of the user's jar to the driver/executors - val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" - - // Internal config to propagate the locations of any extra jars to add to the classpath - // of the executors - val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" - - // Staging directory is private! -> rwx-------- - val STAGING_DIR_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) - - // App files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) - - /** - * Find the user-defined Spark jar if configured, or return the jar containing this - * class if not. - * - * This method first looks in the SparkConf object for the CONF_SPARK_JAR key, and in the - * user environment if that is not found (for backwards compatibility). - */ - private def sparkJar(conf: SparkConf): String = { - if (conf.contains(CONF_SPARK_JAR)) { - conf.get(CONF_SPARK_JAR) - } else if (System.getenv(ENV_SPARK_JAR) != null) { - logWarning( - s"$ENV_SPARK_JAR detected in the system environment. This variable has been deprecated " + - s"in favor of the $CONF_SPARK_JAR configuration variable.") - System.getenv(ENV_SPARK_JAR) - } else { - SparkContext.jarOfClass(this.getClass).head - } - } - - /** - * Return the path to the given application's staging directory. - */ - private def getAppStagingDir(appId: ApplicationId): String = { - SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR - } - - /** - * Populate the classpath entry in the given environment map with any application - * classpath specified through the Hadoop and Yarn configurations. - */ - def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]): Unit = { - val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf) - for (c <- classPathElementsToAdd.flatten) { - YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, c.trim) - } - } - - private def getYarnAppClasspath(conf: Configuration): Option[Seq[String]] = - Option(conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) match { - case Some(s) => Some(s.toSeq) - case None => getDefaultYarnApplicationClasspath - } - - private def getMRAppClasspath(conf: Configuration): Option[Seq[String]] = - Option(conf.getStrings("mapreduce.application.classpath")) match { - case Some(s) => Some(s.toSeq) - case None => getDefaultMRApplicationClasspath - } - - def getDefaultYarnApplicationClasspath: Option[Seq[String]] = { - val triedDefault = Try[Seq[String]] { - val field = classOf[YarnConfiguration].getField("DEFAULT_YARN_APPLICATION_CLASSPATH") - val value = field.get(null).asInstanceOf[Array[String]] - value.toSeq - } recoverWith { - case e: NoSuchFieldException => Success(Seq.empty[String]) - } - - triedDefault match { - case f: Failure[_] => - logError("Unable to obtain the default YARN Application classpath.", f.exception) - case s: Success[_] => - logDebug(s"Using the default YARN application classpath: ${s.get.mkString(",")}") - } - - triedDefault.toOption - } - - /** - * In Hadoop 0.23, the MR application classpath comes with the YARN application - * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. - * So we need to use reflection to retrieve it. - */ - def getDefaultMRApplicationClasspath: Option[Seq[String]] = { - val triedDefault = Try[Seq[String]] { - val field = classOf[MRJobConfig].getField("DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH") - val value = if (field.getType == classOf[String]) { - StringUtils.getStrings(field.get(null).asInstanceOf[String]).toArray - } else { - field.get(null).asInstanceOf[Array[String]] - } - value.toSeq - } recoverWith { - case e: NoSuchFieldException => Success(Seq.empty[String]) - } - - triedDefault match { - case f: Failure[_] => - logError("Unable to obtain the default MR Application classpath.", f.exception) - case s: Success[_] => - logDebug(s"Using the default MR application classpath: ${s.get.mkString(",")}") - } - - triedDefault.toOption - } - - /** - * Populate the classpath entry in the given environment map. - * This includes the user jar, Spark jar, and any extra application jars. - */ - def populateClasspath( - args: ClientArguments, - conf: Configuration, - sparkConf: SparkConf, - env: HashMap[String, String], - extraClassPath: Option[String] = None): Unit = { - extraClassPath.foreach(addClasspathEntry(_, env)) - addClasspathEntry(Environment.PWD.$(), env) - - // Normally the users app.jar is last in case conflicts with spark jars - if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { - addUserClasspath(args, sparkConf, env) - addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - populateHadoopClasspath(conf, env) - } else { - addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - populateHadoopClasspath(conf, env) - addUserClasspath(args, sparkConf, env) - } - - // Append all jar files under the working directory to the classpath. - addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env) - } - - /** - * Adds the user jars which have local: URIs (or alternate names, such as APP_JAR) explicitly - * to the classpath. - */ - private def addUserClasspath( - args: ClientArguments, - conf: SparkConf, - env: HashMap[String, String]): Unit = { - - // If `args` is not null, we are launching an AM container. - // Otherwise, we are launching executor containers. - val (mainJar, secondaryJars) = - if (args != null) { - (args.userJar, args.addJars) - } else { - (conf.get(CONF_SPARK_USER_JAR, null), conf.get(CONF_SPARK_YARN_SECONDARY_JARS, null)) - } - - addFileToClasspath(mainJar, APP_JAR, env) - if (secondaryJars != null) { - secondaryJars.split(",").filter(_.nonEmpty).foreach { jar => - addFileToClasspath(jar, null, env) - } - } - } - - /** - * Adds the given path to the classpath, handling "local:" URIs correctly. - * - * If an alternate name for the file is given, and it's not a "local:" file, the alternate - * name will be added to the classpath (relative to the job's work directory). - * - * If not a "local:" file and no alternate name, the environment is not modified. - * - * @param path Path to add to classpath (optional). - * @param fileName Alternate name for the file (optional). - * @param env Map holding the environment variables. - */ - private def addFileToClasspath( - path: String, - fileName: String, - env: HashMap[String, String]): Unit = { - if (path != null) { - scala.util.control.Exception.ignoring(classOf[URISyntaxException]) { - val uri = new URI(path) - if (uri.getScheme == LOCAL_SCHEME) { - addClasspathEntry(uri.getPath, env) - return - } - } - } - if (fileName != null) { - addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env) - } - } - - /** - * Add the given path to the classpath entry of the given environment map. - * If the classpath is already set, this appends the new path to the existing classpath. - */ - private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit = - YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path) - - /** - * Get the list of namenodes the user may access. - */ - def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { - sparkConf.get("spark.yarn.access.namenodes", "") - .split(",") - .map(_.trim()) - .filter(!_.isEmpty) - .map(new Path(_)) - .toSet - } - - def getTokenRenewer(conf: Configuration): String = { - val delegTokenRenewer = Master.getMasterPrincipal(conf) - logDebug("delegation token renewer is: " + delegTokenRenewer) - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) - } - delegTokenRenewer - } - - /** - * Obtains tokens for the namenodes passed in and adds them to the credentials. - */ - def obtainTokensForNamenodes( - paths: Set[Path], - conf: Configuration, - creds: Credentials): Unit = { - if (UserGroupInformation.isSecurityEnabled()) { - val delegTokenRenewer = getTokenRenewer(conf) - paths.foreach { dst => - val dstFs = dst.getFileSystem(conf) - logDebug("getting token for namenode: " + dst) - dstFs.addDelegationTokens(delegTokenRenewer, creds) - } - } - } - - /** - * Return whether the two file systems are the same. - */ - private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { - val srcUri = srcFs.getUri() - val dstUri = destFs.getUri() - if (srcUri.getScheme() == null || srcUri.getScheme() != dstUri.getScheme()) { - return false - } - - var srcHost = srcUri.getHost() - var dstHost = dstUri.getHost() - - // In HA or when using viewfs, the host part of the URI may not actually be a host, but the - // name of the HDFS namespace. Those names won't resolve, so avoid even trying if they - // match. - if (srcHost != null && dstHost != null && srcHost != dstHost) { - try { - srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() - dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() - } catch { - case e: UnknownHostException => - return false - } - } - - Objects.equal(srcHost, dstHost) && srcUri.getPort() == dstUri.getPort() - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index fdd3c2300fa78..ebf5616e8d303 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -17,32 +17,33 @@ package org.apache.spark.deploy.yarn +import java.net.URI import java.nio.ByteBuffer -import java.security.PrivilegedExceptionAction + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import org.apache.spark.util.Utils import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, ListBuffer} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils -import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.client.api.NMClient import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{SecurityManager, SparkConf, Logging} import org.apache.spark.network.util.JavaUtils - class ExecutorRunnable( container: Container, conf: Configuration, - spConf: SparkConf, + sparkConf: SparkConf, masterAddress: String, slaveId: String, hostname: String, @@ -50,13 +51,13 @@ class ExecutorRunnable( executorCores: Int, appId: String, securityMgr: SecurityManager) - extends Runnable with ExecutorRunnableUtil with Logging { + extends Runnable with Logging { var rpc: YarnRPC = YarnRPC.create(conf) var nmClient: NMClient = _ - val sparkConf = spConf val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - + lazy val env = prepareEnvironment + def run = { logInfo("Starting Executor Container") nmClient = NMClient.createNMClient() @@ -110,4 +111,165 @@ class ExecutorRunnable( nmClient.startContainer(container, ctx) } + private def prepareCommand( + masterAddress: String, + slaveId: String, + hostname: String, + executorMemory: Int, + executorCores: Int, + appId: String, + localResources: HashMap[String, LocalResource]): List[String] = { + // Extra options for the JVM + val javaOpts = ListBuffer[String]() + + // Set the environment variable through a command prefix + // to append to the existing value of the variable + var prefixEnv: Option[String] = None + + // Set the JVM memory + val executorMemoryString = executorMemory + "m" + javaOpts += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " + + // Set extra Java options for the executor, if defined + sys.props.get("spark.executor.extraJavaOptions").foreach { opts => + javaOpts += opts + } + sys.env.get("SPARK_JAVA_OPTS").foreach { opts => + javaOpts += opts + } + sys.props.get("spark.executor.extraLibraryPath").foreach { p => + prefixEnv = Some(Utils.libraryPathEnvPrefix(Seq(p))) + } + + javaOpts += "-Djava.io.tmpdir=" + + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + + // Certain configs need to be passed here because they are needed before the Executor + // registers with the Scheduler and transfers the spark configs. Since the Executor backend + // uses Akka to connect to the scheduler, the akka settings are needed as well as the + // authentication settings. + sparkConf.getAll. + filter { case (k, v) => k.startsWith("spark.auth") || k.startsWith("spark.akka") }. + foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } + + sparkConf.getAkkaConf. + foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } + + // Commenting it out for now - so that people can refer to the properties if required. Remove + // it once cpuset version is pushed out. + // The context is, default gc for server class machines end up using all cores to do gc - hence + // if there are multiple containers in same node, spark gc effects all other containers + // performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in + // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset + // of cores on a node. + /* + else { + // If no java_opts specified, default to using -XX:+CMSIncrementalMode + // It might be possible that other modes/config is being done in + // spark.executor.extraJavaOptions, so we dont want to mess with it. + // In our expts, using (default) throughput collector has severe perf ramnifications in + // multi-tennent machines + // The options are based on + // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use + // %20the%20Concurrent%20Low%20Pause%20Collector|outline + javaOpts += " -XX:+UseConcMarkSweepGC " + javaOpts += " -XX:+CMSIncrementalMode " + javaOpts += " -XX:+CMSIncrementalPacing " + javaOpts += " -XX:CMSIncrementalDutyCycleMin=0 " + javaOpts += " -XX:CMSIncrementalDutyCycle=10 " + } + */ + + // For log4j configuration to reference + javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) + + val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", + "-server", + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. + // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in + // an inconsistent state. + // TODO: If the OOM is not recoverable by rescheduling it on different node, then do + // 'something' to fail job ... akin to blacklisting trackers in mapred ? + "-XX:OnOutOfMemoryError='kill %p'") ++ + javaOpts ++ + Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", + masterAddress.toString, + slaveId.toString, + hostname.toString, + executorCores.toString, + appId, + "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", + "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + // TODO: it would be nicer to just make sure there are no null commands here + commands.map(s => if (s == null) "null" else s).toList + } + + private def setupDistributedCache( + file: String, + rtype: LocalResourceType, + localResources: HashMap[String, LocalResource], + timestamp: String, + size: String, + vis: String): Unit = { + val uri = new URI(file) + val amJarRsrc = Records.newRecord(classOf[LocalResource]) + amJarRsrc.setType(rtype) + amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) + amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) + amJarRsrc.setTimestamp(timestamp.toLong) + amJarRsrc.setSize(size.toLong) + localResources(uri.getFragment()) = amJarRsrc + } + + private def prepareLocalResources: HashMap[String, LocalResource] = { + logInfo("Preparing Local resources") + val localResources = HashMap[String, LocalResource]() + + if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { + val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') + val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') + val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',') + val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',') + for( i <- 0 to distFiles.length - 1) { + setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i), + fileSizes(i), visibilities(i)) + } + } + + if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) { + val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',') + val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',') + val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',') + val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',') + for( i <- 0 to distArchives.length - 1) { + setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, + timeStamps(i), fileSizes(i), visibilities(i)) + } + } + + logInfo("Prepared Local resources " + localResources) + localResources + } + + private def prepareEnvironment: HashMap[String, String] = { + val env = new HashMap[String, String]() + val extraCp = sparkConf.getOption("spark.executor.extraClassPath") + Client.populateClasspath(null, yarnConf, sparkConf, env, extraCp) + + sparkConf.getExecutorEnv.foreach { case (key, value) => + // This assumes each executor environment variable set here is a path + // This is kept for backward compatibility and consistency with hadoop + YarnSparkHadoopUtil.addPathToEnvironment(env, key, value) + } + + // Keep this for backwards compatibility but users should move to the config + sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) + } + + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k, v) => env(k) = v } + env + } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala deleted file mode 100644 index 22d73ecf6d010..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ /dev/null @@ -1,203 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.net.URI - -import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, ListBuffer} - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.{ConverterUtils, Records} - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.Utils - -trait ExecutorRunnableUtil extends Logging { - - val yarnConf: YarnConfiguration - val sparkConf: SparkConf - lazy val env = prepareEnvironment - - def prepareCommand( - masterAddress: String, - slaveId: String, - hostname: String, - executorMemory: Int, - executorCores: Int, - appId: String, - localResources: HashMap[String, LocalResource]): List[String] = { - // Extra options for the JVM - val javaOpts = ListBuffer[String]() - - // Set the environment variable through a command prefix - // to append to the existing value of the variable - var prefixEnv: Option[String] = None - - // Set the JVM memory - val executorMemoryString = executorMemory + "m" - javaOpts += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " - - // Set extra Java options for the executor, if defined - sys.props.get("spark.executor.extraJavaOptions").foreach { opts => - javaOpts += opts - } - sys.env.get("SPARK_JAVA_OPTS").foreach { opts => - javaOpts += opts - } - sys.props.get("spark.executor.extraLibraryPath").foreach { p => - prefixEnv = Some(Utils.libraryPathEnvPrefix(Seq(p))) - } - - javaOpts += "-Djava.io.tmpdir=" + - new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) - - // Certain configs need to be passed here because they are needed before the Executor - // registers with the Scheduler and transfers the spark configs. Since the Executor backend - // uses Akka to connect to the scheduler, the akka settings are needed as well as the - // authentication settings. - sparkConf.getAll. - filter { case (k, v) => k.startsWith("spark.auth") || k.startsWith("spark.akka") }. - foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } - - sparkConf.getAkkaConf. - foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } - - // Commenting it out for now - so that people can refer to the properties if required. Remove - // it once cpuset version is pushed out. - // The context is, default gc for server class machines end up using all cores to do gc - hence - // if there are multiple containers in same node, spark gc effects all other containers - // performance (which can also be other spark containers) - // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in - // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset - // of cores on a node. - /* - else { - // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in - // spark.executor.extraJavaOptions, so we dont want to mess with it. - // In our expts, using (default) throughput collector has severe perf ramnifications in - // multi-tennent machines - // The options are based on - // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use - // %20the%20Concurrent%20Low%20Pause%20Collector|outline - javaOpts += " -XX:+UseConcMarkSweepGC " - javaOpts += " -XX:+CMSIncrementalMode " - javaOpts += " -XX:+CMSIncrementalPacing " - javaOpts += " -XX:CMSIncrementalDutyCycleMin=0 " - javaOpts += " -XX:CMSIncrementalDutyCycle=10 " - } - */ - - // For log4j configuration to reference - javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) - - val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", - "-server", - // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. - // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in - // an inconsistent state. - // TODO: If the OOM is not recoverable by rescheduling it on different node, then do - // 'something' to fail job ... akin to blacklisting trackers in mapred ? - "-XX:OnOutOfMemoryError='kill %p'") ++ - javaOpts ++ - Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", - masterAddress.toString, - slaveId.toString, - hostname.toString, - executorCores.toString, - appId, - "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", - "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - - // TODO: it would be nicer to just make sure there are no null commands here - commands.map(s => if (s == null) "null" else s).toList - } - - private def setupDistributedCache( - file: String, - rtype: LocalResourceType, - localResources: HashMap[String, LocalResource], - timestamp: String, - size: String, - vis: String): Unit = { - val uri = new URI(file) - val amJarRsrc = Records.newRecord(classOf[LocalResource]) - amJarRsrc.setType(rtype) - amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) - amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) - amJarRsrc.setTimestamp(timestamp.toLong) - amJarRsrc.setSize(size.toLong) - localResources(uri.getFragment()) = amJarRsrc - } - - def prepareLocalResources: HashMap[String, LocalResource] = { - logInfo("Preparing Local resources") - val localResources = HashMap[String, LocalResource]() - - if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { - val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') - val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') - val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',') - val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',') - for( i <- 0 to distFiles.length - 1) { - setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i), - fileSizes(i), visibilities(i)) - } - } - - if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) { - val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',') - val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',') - val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',') - val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',') - for( i <- 0 to distArchives.length - 1) { - setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, - timeStamps(i), fileSizes(i), visibilities(i)) - } - } - - logInfo("Prepared Local resources " + localResources) - localResources - } - - def prepareEnvironment: HashMap[String, String] = { - val env = new HashMap[String, String]() - val extraCp = sparkConf.getOption("spark.executor.extraClassPath") - ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp) - - sparkConf.getExecutorEnv.foreach { case (key, value) => - // This assumes each executor environment variable set here is a path - // This is kept for backward compatibility and consistency with hadoop - YarnSparkHadoopUtil.addPathToEnvironment(env, key, value) - } - - // Keep this for backwards compatibility but users should move to the config - sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => - YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) - } - - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k, v) => env(k) = v } - env - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala deleted file mode 100644 index 2bbf5d7db8668..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ /dev/null @@ -1,213 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap} - -import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.scheduler.SplitInfo - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.util.Records - -/** - * Acquires resources for executors from a ResourceManager and launches executors in new containers. - */ -private[yarn] class YarnAllocationHandler( - conf: Configuration, - sparkConf: SparkConf, - amClient: AMRMClient[ContainerRequest], - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]], - securityMgr: SecurityManager) - extends YarnAllocator(conf, sparkConf, appAttemptId, args, preferredNodes, securityMgr) { - - override protected def releaseContainer(container: Container) = { - amClient.releaseAssignedContainer(container.getId()) - } - - // pending isn't used on stable as the AMRMClient handles incremental asks - override protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse = { - addResourceRequests(count) - - // We have already set the container request. Poll the ResourceManager for a response. - // This doubles as a heartbeat if there are no pending container requests. - val progressIndicator = 0.1f - new StableAllocateResponse(amClient.allocate(progressIndicator)) - } - - private def createRackResourceRequests( - hostContainers: ArrayBuffer[ContainerRequest] - ): ArrayBuffer[ContainerRequest] = { - // Generate modified racks and new set of hosts under it before issuing requests. - val rackToCounts = new HashMap[String, Int]() - - for (container <- hostContainers) { - val candidateHost = container.getNodes.last - assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - if (rack != null) { - var count = rackToCounts.getOrElse(rack, 0) - count += 1 - rackToCounts.put(rack, count) - } - } - - val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size) - for ((rack, count) <- rackToCounts) { - requestedContainers ++= createResourceRequests( - AllocationType.RACK, - rack, - count, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - - requestedContainers - } - - private def addResourceRequests(numExecutors: Int) { - val containerRequests: List[ContainerRequest] = - if (numExecutors <= 0) { - logDebug("numExecutors: " + numExecutors) - List() - } else if (preferredHostToCount.isEmpty) { - logDebug("host preferences is empty") - createResourceRequests( - AllocationType.ANY, - resource = null, - numExecutors, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY).toList - } else { - // Request for all hosts in preferred nodes and for numExecutors - - // candidates.size, request by default allocation policy. - val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) - for ((candidateHost, candidateCount) <- preferredHostToCount) { - val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) - - if (requiredCount > 0) { - hostContainerRequests ++= createResourceRequests( - AllocationType.HOST, - candidateHost, - requiredCount, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - } - val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( - hostContainerRequests).toList - - val anyContainerRequests = createResourceRequests( - AllocationType.ANY, - resource = null, - numExecutors, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - - val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( - hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size) - - containerRequestBuffer ++= hostContainerRequests - containerRequestBuffer ++= rackContainerRequests - containerRequestBuffer ++= anyContainerRequests - containerRequestBuffer.toList - } - - for (request <- containerRequests) { - amClient.addContainerRequest(request) - } - - for (request <- containerRequests) { - val nodes = request.getNodes - var hostStr = if (nodes == null || nodes.isEmpty) { - "Any" - } else { - nodes.last - } - logInfo("Container request (host: %s, priority: %s, capability: %s".format( - hostStr, - request.getPriority().getPriority, - request.getCapability)) - } - } - - private def createResourceRequests( - requestType: AllocationType.AllocationType, - resource: String, - numExecutors: Int, - priority: Int - ): ArrayBuffer[ContainerRequest] = { - - // If hostname is specified, then we need at least two requests - node local and rack local. - // There must be a third request, which is ANY. That will be specially handled. - requestType match { - case AllocationType.HOST => { - assert(YarnSparkHadoopUtil.ANY_HOST != resource) - val hostname = resource - val nodeLocal = constructContainerRequests( - Array(hostname), - racks = null, - numExecutors, - priority) - - // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. - YarnSparkHadoopUtil.populateRackInfo(conf, hostname) - nodeLocal - } - case AllocationType.RACK => { - val rack = resource - constructContainerRequests(hosts = null, Array(rack), numExecutors, priority) - } - case AllocationType.ANY => constructContainerRequests( - hosts = null, racks = null, numExecutors, priority) - case _ => throw new IllegalArgumentException( - "Unexpected/unsupported request type: " + requestType) - } - } - - private def constructContainerRequests( - hosts: Array[String], - racks: Array[String], - numExecutors: Int, - priority: Int - ): ArrayBuffer[ContainerRequest] = { - - val memoryRequest = executorMemory + memoryOverhead - val resource = Resource.newInstance(memoryRequest, executorCores) - - val prioritySetting = Records.newRecord(classOf[Priority]) - prioritySetting.setPriority(priority) - - val requests = new ArrayBuffer[ContainerRequest]() - for (i <- 0 until numExecutors) { - requests += new ContainerRequest(resource, hosts, racks, prioritySetting) - } - requests - } - - private class StableAllocateResponse(response: AllocateResponse) extends YarnAllocateResponse { - override def getAllocatedContainers() = response.getAllocatedContainers() - override def getAvailableResources() = response.getAvailableResources() - override def getCompletedContainersStatuses() = response.getCompletedContainersStatuses() - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index b32e15738f28b..de65ef23ad1ce 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -17,7 +17,6 @@ package org.apache.spark.deploy.yarn -import java.util.{List => JList} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import java.util.regex.Pattern @@ -25,17 +24,20 @@ import java.util.regex.Pattern import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import com.google.common.util.concurrent.ThreadFactoryBuilder + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ - object AllocationType extends Enumeration { type AllocationType = Value val HOST, RACK, ANY = Value @@ -52,12 +54,12 @@ object AllocationType extends Enumeration { // more info on how we are requesting for containers. /** - * Common code for the Yarn container allocator. Contains all the version-agnostic code to - * manage container allocation for a running Spark application. + * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ -private[yarn] abstract class YarnAllocator( +private[yarn] class YarnAllocator( conf: Configuration, sparkConf: SparkConf, + amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], @@ -67,7 +69,7 @@ private[yarn] abstract class YarnAllocator( import YarnAllocator._ // These three are locked on allocatedHostToContainersMap. Complementary data structures - // allocatedHostToContainersMap : containers which are running : host, Set + // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. private val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]() @@ -161,8 +163,6 @@ private[yarn] abstract class YarnAllocator( def allocateResources(): Unit = synchronized { val missing = maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get() - // this is needed by alpha, do it here since we add numPending right after this - val executorsPending = numPendingAllocate.get() if (missing > 0) { val totalExecutorMemory = executorMemory + memoryOverhead numPendingAllocate.addAndGet(missing) @@ -172,7 +172,7 @@ private[yarn] abstract class YarnAllocator( logDebug("Empty allocation request ...") } - val allocateResponse = allocateContainers(missing, executorsPending) + val allocateResponse = allocateContainers(missing) val allocatedContainers = allocateResponse.getAllocatedContainers() if (allocatedContainers.size > 0) { @@ -368,7 +368,7 @@ private[yarn] abstract class YarnAllocator( val containerId = completedContainer.getContainerId if (releasedContainers.containsKey(containerId)) { - // YarnAllocationHandler already marked the container for release, so remove it from + // Already marked the container for release, so remove it from // `releasedContainers`. releasedContainers.remove(containerId) } else { @@ -441,20 +441,16 @@ private[yarn] abstract class YarnAllocator( } } - protected def allocatedContainersOnHost(host: String): Int = { - var retval = 0 + private def allocatedContainersOnHost(host: String): Int = { allocatedHostToContainersMap.synchronized { - retval = allocatedHostToContainersMap.getOrElse(host, Set()).size + allocatedHostToContainersMap.getOrElse(host, Set()).size } - retval } - protected def allocatedContainersOnRack(rack: String): Int = { - var retval = 0 + private def allocatedContainersOnRack(rack: String): Int = { allocatedHostToContainersMap.synchronized { - retval = allocatedRackCount.getOrElse(rack, 0) + allocatedRackCount.getOrElse(rack, 0) } - retval } private def isResourceConstraintSatisfied(container: Container): Boolean = { @@ -464,9 +460,8 @@ private[yarn] abstract class YarnAllocator( // A simple method to copy the split info map. private def generateNodeToWeight( conf: Configuration, - input: collection.Map[String, collection.Set[SplitInfo]] - ): (Map[String, Int], Map[String, Int]) = { - + input: collection.Map[String, collection.Set[SplitInfo]]) + : (Map[String, Int], Map[String, Int]) = { if (input == null) { return (Map[String, Int](), Map[String, Int]()) } @@ -488,9 +483,9 @@ private[yarn] abstract class YarnAllocator( (hostToCount.toMap, rackToCount.toMap) } - private def internalReleaseContainer(container: Container) = { + private def internalReleaseContainer(container: Container): Unit = { releasedContainers.put(container.getId(), true) - releaseContainer(container) + amClient.releaseAssignedContainer(container.getId()) } /** @@ -498,26 +493,158 @@ private[yarn] abstract class YarnAllocator( * * @param count Number of containers to allocate. * If zero, should still contact RM (as a heartbeat). - * @param pending Number of containers pending allocate. Only used on alpha. * @return Response to the allocation request. */ - protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse + private def allocateContainers(count: Int): AllocateResponse = { + addResourceRequests(count) + + // We have already set the container request. Poll the ResourceManager for a response. + // This doubles as a heartbeat if there are no pending container requests. + val progressIndicator = 0.1f + amClient.allocate(progressIndicator) + } - /** Called to release a previously allocated container. */ - protected def releaseContainer(container: Container): Unit + private def createRackResourceRequests(hostContainers: ArrayBuffer[ContainerRequest]) + : ArrayBuffer[ContainerRequest] = { + // Generate modified racks and new set of hosts under it before issuing requests. + val rackToCounts = new HashMap[String, Int]() - /** - * Defines the interface for an allocate response from the RM. This is needed since the alpha - * and stable interfaces differ here in ways that cannot be fixed using other routes. - */ - protected trait YarnAllocateResponse { + for (container <- hostContainers) { + val candidateHost = container.getNodes.last + assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) + + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) + if (rack != null) { + var count = rackToCounts.getOrElse(rack, 0) + count += 1 + rackToCounts.put(rack, count) + } + } + + val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size) + for ((rack, count) <- rackToCounts) { + requestedContainers ++= createResourceRequests( + AllocationType.RACK, + rack, + count, + RM_REQUEST_PRIORITY) + } + + requestedContainers + } + + private def addResourceRequests(numExecutors: Int): Unit = { + val containerRequests: List[ContainerRequest] = + if (numExecutors <= 0) { + logDebug("numExecutors: " + numExecutors) + List() + } else if (preferredHostToCount.isEmpty) { + logDebug("host preferences is empty") + createResourceRequests( + AllocationType.ANY, + resource = null, + numExecutors, + RM_REQUEST_PRIORITY).toList + } else { + // Request for all hosts in preferred nodes and for numExecutors - + // candidates.size, request by default allocation policy. + val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) + for ((candidateHost, candidateCount) <- preferredHostToCount) { + val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) + + if (requiredCount > 0) { + hostContainerRequests ++= createResourceRequests( + AllocationType.HOST, + candidateHost, + requiredCount, + RM_REQUEST_PRIORITY) + } + } + val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( + hostContainerRequests).toList + + val anyContainerRequests = createResourceRequests( + AllocationType.ANY, + resource = null, + numExecutors, + RM_REQUEST_PRIORITY) + + val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( + hostContainerRequests.size + rackContainerRequests.size + anyContainerRequests.size) + + containerRequestBuffer ++= hostContainerRequests + containerRequestBuffer ++= rackContainerRequests + containerRequestBuffer ++= anyContainerRequests + containerRequestBuffer.toList + } - def getAllocatedContainers(): JList[Container] + for (request <- containerRequests) { + amClient.addContainerRequest(request) + } - def getAvailableResources(): Resource + for (request <- containerRequests) { + val nodes = request.getNodes + val hostStr = if (nodes == null || nodes.isEmpty) { + "Any" + } else { + nodes.last + } + logInfo("Container request (host: %s, priority: %s, capability: %s".format( + hostStr, + request.getPriority().getPriority, + request.getCapability)) + } + } - def getCompletedContainersStatuses(): JList[ContainerStatus] + private def createResourceRequests( + requestType: AllocationType.AllocationType, + resource: String, + numExecutors: Int, + priority: Int): ArrayBuffer[ContainerRequest] = { + // If hostname is specified, then we need at least two requests - node local and rack local. + // There must be a third request, which is ANY. That will be specially handled. + requestType match { + case AllocationType.HOST => { + assert(YarnSparkHadoopUtil.ANY_HOST != resource) + val hostname = resource + val nodeLocal = constructContainerRequests( + Array(hostname), + racks = null, + numExecutors, + priority) + + // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. + YarnSparkHadoopUtil.populateRackInfo(conf, hostname) + nodeLocal + } + case AllocationType.RACK => { + val rack = resource + constructContainerRequests(hosts = null, Array(rack), numExecutors, priority) + } + case AllocationType.ANY => constructContainerRequests( + hosts = null, racks = null, numExecutors, priority) + case _ => throw new IllegalArgumentException( + "Unexpected/unsupported request type: " + requestType) + } + } + private def constructContainerRequests( + hosts: Array[String], + racks: Array[String], + numExecutors: Int, + priority: Int + ): ArrayBuffer[ContainerRequest] = { + val memoryRequest = executorMemory + memoryOverhead + val resource = Resource.newInstance(memoryRequest, executorCores) + + val prioritySetting = Records.newRecord(classOf[Priority]) + prioritySetting.setPriority(priority) + + val requests = new ArrayBuffer[ContainerRequest]() + for (i <- 0 until numExecutors) { + requests += new ContainerRequest(resource, hosts, racks, prioritySetting) + } + requests } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 2510b9c9cef68..b45e599588ad3 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -17,19 +17,33 @@ package org.apache.spark.deploy.yarn +import java.util.{List => JList} + +import scala.collection.JavaConversions._ import scala.collection.{Map, Set} +import scala.util.Try -import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.ConverterUtils +import org.apache.hadoop.yarn.webapp.util.WebAppUtils -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo +import org.apache.spark.util.Utils /** - * Interface that defines a Yarn RM client. Abstracts away Yarn version-specific functionality that - * is used by Spark's AM. + * Handles registering and unregistering the application with the YARN ResourceManager. */ -trait YarnRMClient { +private[spark] class YarnRMClient(args: ApplicationMasterArguments) extends Logging { + + private var amClient: AMRMClient[ContainerRequest] = _ + private var uiHistoryAddress: String = _ + private var registered: Boolean = false /** * Registers the application master with the RM. @@ -46,7 +60,21 @@ trait YarnRMClient { preferredNodeLocations: Map[String, Set[SplitInfo]], uiAddress: String, uiHistoryAddress: String, - securityMgr: SecurityManager): YarnAllocator + securityMgr: SecurityManager + ): YarnAllocator = { + amClient = AMRMClient.createAMRMClient() + amClient.init(conf) + amClient.start() + this.uiHistoryAddress = uiHistoryAddress + + logInfo("Registering the ApplicationMaster") + synchronized { + amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + registered = true + } + new YarnAllocator(conf, sparkConf, amClient, getAttemptId(), args, + preferredNodeLocations, securityMgr) + } /** * Unregister the AM. Guaranteed to only be called once. @@ -54,15 +82,54 @@ trait YarnRMClient { * @param status The final status of the AM. * @param diagnostics Diagnostics message to include in the final status. */ - def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit + def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit = synchronized { + if (registered) { + amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) + } + } /** Returns the attempt ID. */ - def getAttemptId(): ApplicationAttemptId + def getAttemptId(): ApplicationAttemptId = { + val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) + val containerId = ConverterUtils.toContainerId(containerIdString) + containerId.getApplicationAttemptId() + } /** Returns the configuration for the AmIpFilter to add to the Spark UI. */ - def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] + def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] = { + // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, + // so not all stable releases have it. + val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration]) + .invoke(null, conf).asInstanceOf[String]).getOrElse("http://") + + // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses. + try { + val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter", + classOf[Configuration]) + val proxies = method.invoke(null, conf).asInstanceOf[JList[String]] + val hosts = proxies.map { proxy => proxy.split(":")(0) } + val uriBases = proxies.map { proxy => prefix + proxy + proxyBase } + Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) + } catch { + case e: NoSuchMethodException => + val proxy = WebAppUtils.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val uriBase = prefix + proxy + proxyBase + Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) + } + } /** Returns the maximum number of attempts to register the AM. */ - def getMaxRegAttempts(conf: YarnConfiguration): Int + def getMaxRegAttempts(sparkConf: SparkConf, yarnConf: YarnConfiguration): Int = { + val sparkMaxAttempts = sparkConf.getOption("spark.yarn.maxAppAttempts").map(_.toInt) + val yarnMaxAttempts = yarnConf.getInt( + YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) + val retval: Int = sparkMaxAttempts match { + case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts + case None => yarnMaxAttempts + } + + retval + } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala deleted file mode 100644 index 8d4b96ed79933..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.util.{List => JList} - -import scala.collection.{Map, Set} -import scala.collection.JavaConversions._ -import scala.util._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.ConverterUtils -import org.apache.hadoop.yarn.webapp.util.WebAppUtils - -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.scheduler.SplitInfo -import org.apache.spark.util.Utils - - -/** - * YarnRMClient implementation for the Yarn stable API. - */ -private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMClient with Logging { - - private var amClient: AMRMClient[ContainerRequest] = _ - private var uiHistoryAddress: String = _ - private var registered: Boolean = false - - override def register( - conf: YarnConfiguration, - sparkConf: SparkConf, - preferredNodeLocations: Map[String, Set[SplitInfo]], - uiAddress: String, - uiHistoryAddress: String, - securityMgr: SecurityManager) = { - amClient = AMRMClient.createAMRMClient() - amClient.init(conf) - amClient.start() - this.uiHistoryAddress = uiHistoryAddress - - logInfo("Registering the ApplicationMaster") - synchronized { - amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) - registered = true - } - new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args, - preferredNodeLocations, securityMgr) - } - - override def unregister(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { - if (registered) { - amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) - } - } - - override def getAttemptId() = { - val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) - val containerId = ConverterUtils.toContainerId(containerIdString) - val appAttemptId = containerId.getApplicationAttemptId() - appAttemptId - } - - override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { - // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, - // so not all stable releases have it. - val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration]) - .invoke(null, conf).asInstanceOf[String]).getOrElse("http://") - - // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses. - try { - val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter", - classOf[Configuration]) - val proxies = method.invoke(null, conf).asInstanceOf[JList[String]] - val hosts = proxies.map { proxy => proxy.split(":")(0) } - val uriBases = proxies.map { proxy => prefix + proxy + proxyBase } - Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) - } catch { - case e: NoSuchMethodException => - val proxy = WebAppUtils.getProxyHostAndPort(conf) - val parts = proxy.split(":") - val uriBase = prefix + proxy + proxyBase - Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) - } - } - - override def getMaxRegAttempts(conf: YarnConfiguration) = - conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) - -} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 2923e6729cd6b..09597bd0e6ab9 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -65,7 +65,8 @@ private[spark] class YarnClientSchedulerBackend( */ private def getExtraClientArguments: Seq[String] = { val extraArgs = new ArrayBuffer[String] - val optionTuples = // List of (target Client argument, environment variable, Spark property) + // List of (target Client argument, environment variable, Spark property) + val optionTuples = List( ("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), @@ -78,11 +79,25 @@ private[spark] class YarnClientSchedulerBackend( ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), ("--name", "SPARK_YARN_APP_NAME", "spark.app.name") ) + // Warn against the following deprecated environment variables: env var -> suggestion + val deprecatedEnvVars = Map( + "SPARK_MASTER_MEMORY" -> "SPARK_DRIVER_MEMORY or --driver-memory through spark-submit", + "SPARK_WORKER_INSTANCES" -> "SPARK_WORKER_INSTANCES or --num-executors through spark-submit", + "SPARK_WORKER_MEMORY" -> "SPARK_EXECUTOR_MEMORY or --executor-memory through spark-submit", + "SPARK_WORKER_CORES" -> "SPARK_EXECUTOR_CORES or --executor-cores through spark-submit") + // Do the same for deprecated properties: property -> suggestion + val deprecatedProps = Map("spark.master.memory" -> "--driver-memory through spark-submit") optionTuples.foreach { case (optionName, envVar, sparkProp) => if (System.getenv(envVar) != null) { extraArgs += (optionName, System.getenv(envVar)) + if (deprecatedEnvVars.contains(envVar)) { + logWarning(s"NOTE: $envVar is deprecated. Use ${deprecatedEnvVars(envVar)} instead.") + } } else if (sc.getConf.contains(sparkProp)) { extraArgs += (optionName, sc.getConf.get(sparkProp)) + if (deprecatedProps.contains(sparkProp)) { + logWarning(s"NOTE: $sparkProp is deprecated. Use ${deprecatedProps(sparkProp)} instead.") + } } } extraArgs diff --git a/yarn/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties index 9dd05f17f012b..287c8e3563503 100644 --- a/yarn/src/test/resources/log4j.properties +++ b/yarn/src/test/resources/log4j.properties @@ -15,10 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file core/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/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala similarity index 76% rename from yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 17b79ae1d82c4..aad50015b717f 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -41,38 +41,38 @@ import scala.util.Try import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.util.Utils -class ClientBaseSuite extends FunSuite with Matchers { +class ClientSuite extends FunSuite with Matchers { test("default Yarn application classpath") { - ClientBase.getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP)) + Client.getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP)) } test("default MR application classpath") { - ClientBase.getDefaultMRApplicationClasspath should be(Some(Fixtures.knownDefMRAppCP)) + Client.getDefaultMRApplicationClasspath should be(Some(Fixtures.knownDefMRAppCP)) } test("resultant classpath for an application that defines a classpath for YARN") { withAppConf(Fixtures.mapYARNAppConf) { conf => val env = newEnv - ClientBase.populateHadoopClasspath(conf, env) + Client.populateHadoopClasspath(conf, env) classpath(env) should be( - flatten(Fixtures.knownYARNAppCP, ClientBase.getDefaultMRApplicationClasspath)) + flatten(Fixtures.knownYARNAppCP, Client.getDefaultMRApplicationClasspath)) } } test("resultant classpath for an application that defines a classpath for MR") { withAppConf(Fixtures.mapMRAppConf) { conf => val env = newEnv - ClientBase.populateHadoopClasspath(conf, env) + Client.populateHadoopClasspath(conf, env) classpath(env) should be( - flatten(ClientBase.getDefaultYarnApplicationClasspath, Fixtures.knownMRAppCP)) + flatten(Client.getDefaultYarnApplicationClasspath, Fixtures.knownMRAppCP)) } } test("resultant classpath for an application that defines both classpaths, YARN and MR") { withAppConf(Fixtures.mapAppConf) { conf => val env = newEnv - ClientBase.populateHadoopClasspath(conf, env) + Client.populateHadoopClasspath(conf, env) classpath(env) should be(flatten(Fixtures.knownYARNAppCP, Fixtures.knownMRAppCP)) } } @@ -83,16 +83,16 @@ class ClientBaseSuite extends FunSuite with Matchers { test("Local jar URIs") { val conf = new Configuration() - val sparkConf = new SparkConf().set(ClientBase.CONF_SPARK_JAR, SPARK) + val sparkConf = new SparkConf().set(Client.CONF_SPARK_JAR, SPARK) val env = new MutableHashMap[String, String]() val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - ClientBase.populateClasspath(args, conf, sparkConf, env) + Client.populateClasspath(args, conf, sparkConf, env) val cp = env("CLASSPATH").split(File.pathSeparator) s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => val uri = new URI(entry) - if (ClientBase.LOCAL_SCHEME.equals(uri.getScheme())) { + if (Client.LOCAL_SCHEME.equals(uri.getScheme())) { cp should contain (uri.getPath()) } else { cp should not contain (uri.getPath()) @@ -100,31 +100,30 @@ class ClientBaseSuite extends FunSuite with Matchers { }) cp should contain (Environment.PWD.$()) cp should contain (s"${Environment.PWD.$()}${File.separator}*") - cp should not contain (ClientBase.SPARK_JAR) - cp should not contain (ClientBase.APP_JAR) + cp should not contain (Client.SPARK_JAR) + cp should not contain (Client.APP_JAR) } test("Jar path propagation through SparkConf") { val conf = new Configuration() - val sparkConf = new SparkConf().set(ClientBase.CONF_SPARK_JAR, SPARK) - val yarnConf = new YarnConfiguration() + val sparkConf = new SparkConf().set(Client.CONF_SPARK_JAR, SPARK) val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) + val client = spy(new Client(args, conf, sparkConf)) doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), any(classOf[Path]), anyShort(), anyBoolean()) val tempDir = Utils.createTempDir() try { client.prepareLocalResources(tempDir.getAbsolutePath()) - sparkConf.getOption(ClientBase.CONF_SPARK_USER_JAR) should be (Some(USER)) + sparkConf.getOption(Client.CONF_SPARK_USER_JAR) should be (Some(USER)) // The non-local path should be propagated by name only, since it will end up in the app's // staging dir. val expected = ADDED.split(",") .map(p => { val uri = new URI(p) - if (ClientBase.LOCAL_SCHEME == uri.getScheme()) { + if (Client.LOCAL_SCHEME == uri.getScheme()) { p } else { Option(uri.getFragment()).getOrElse(new File(p).getName()) @@ -132,7 +131,7 @@ class ClientBaseSuite extends FunSuite with Matchers { }) .mkString(",") - sparkConf.getOption(ClientBase.CONF_SPARK_YARN_SECONDARY_JARS) should be (Some(expected)) + sparkConf.getOption(Client.CONF_SPARK_YARN_SECONDARY_JARS) should be (Some(expected)) } finally { Utils.deleteRecursively(tempDir) } @@ -141,34 +140,34 @@ class ClientBaseSuite extends FunSuite with Matchers { test("check access nns empty") { val sparkConf = new SparkConf() sparkConf.set("spark.yarn.access.namenodes", "") - val nns = ClientBase.getNameNodesToAccess(sparkConf) + val nns = Client.getNameNodesToAccess(sparkConf) nns should be(Set()) } test("check access nns unset") { val sparkConf = new SparkConf() - val nns = ClientBase.getNameNodesToAccess(sparkConf) + val nns = Client.getNameNodesToAccess(sparkConf) nns should be(Set()) } test("check access nns") { val sparkConf = new SparkConf() sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032") - val nns = ClientBase.getNameNodesToAccess(sparkConf) + val nns = Client.getNameNodesToAccess(sparkConf) nns should be(Set(new Path("hdfs://nn1:8032"))) } test("check access nns space") { val sparkConf = new SparkConf() sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032, ") - val nns = ClientBase.getNameNodesToAccess(sparkConf) + val nns = Client.getNameNodesToAccess(sparkConf) nns should be(Set(new Path("hdfs://nn1:8032"))) } test("check access two nns") { val sparkConf = new SparkConf() sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032,hdfs://nn2:8032") - val nns = ClientBase.getNameNodesToAccess(sparkConf) + val nns = Client.getNameNodesToAccess(sparkConf) nns should be(Set(new Path("hdfs://nn1:8032"), new Path("hdfs://nn2:8032"))) } @@ -176,7 +175,7 @@ class ClientBaseSuite extends FunSuite with Matchers { val hadoopConf = new Configuration() hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") - val renewer = ClientBase.getTokenRenewer(hadoopConf) + val renewer = Client.getTokenRenewer(hadoopConf) renewer should be ("yarn/myrm:8032@SPARKTEST.COM") } @@ -184,7 +183,7 @@ class ClientBaseSuite extends FunSuite with Matchers { val hadoopConf = new Configuration() val caught = intercept[SparkException] { - ClientBase.getTokenRenewer(hadoopConf) + Client.getTokenRenewer(hadoopConf) } assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") } @@ -218,7 +217,7 @@ class ClientBaseSuite extends FunSuite with Matchers { def withAppConf(m: Map[String, String] = Map())(testCode: (Configuration) => Any) { val conf = new Configuration - m.foreach { case (k, v) => conf.set(k, v, "ClientBaseSpec") } + m.foreach { case (k, v) => conf.set(k, v, "ClientSpec") } testCode(conf) } @@ -242,15 +241,4 @@ class ClientBaseSuite extends FunSuite with Matchers { }.toOption.getOrElse(defaults) } - private class DummyClient( - val args: ClientArguments, - val hadoopConf: Configuration, - val sparkConf: SparkConf, - val yarnConf: YarnConfiguration) extends ClientBase { - override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = ??? - override def submitApplication(): ApplicationId = ??? - override def getApplicationReport(appId: ApplicationId): ApplicationReport = ??? - override def getClientToken(report: ApplicationReport): String = ??? - } - }