diff --git a/.gitignore b/.gitignore index cd9f90d55932c..857e9feb953bd 100644 --- a/.gitignore +++ b/.gitignore @@ -18,6 +18,7 @@ conf/java-opts conf/spark-env.sh conf/streaming-env.sh conf/log4j.properties +conf/spark-defaults.conf docs/_site docs/api target/ diff --git a/.rat-excludes b/.rat-excludes index 9e9abb3f10bbf..50766954ef070 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -11,6 +11,7 @@ RELEASE control docs fairscheduler.xml.template +spark-defaults.conf.template log4j.properties log4j.properties.template metrics.properties.template diff --git a/assembly/pom.xml b/assembly/pom.xml index 923bf47f7076a..bdb38806492a6 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -33,7 +33,7 @@ scala-${scala.binary.version} - ${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + spark-assembly-${project.version}-hadoop${hadoop.version}.jar ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} spark /usr/share/spark diff --git a/bagel/pom.xml b/bagel/pom.xml index 142f75c5d2c64..355f437c5b16a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -31,20 +31,6 @@ Spark Project Bagel http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 4f60bff19cb93..065553eb31939 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -1,69 +1,88 @@ -@echo off - -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 This script computes Spark's classpath and prints it to stdout; it's used by both the "run" -rem script and the ExecutorRunner in standalone cluster mode. - -set SCALA_VERSION=2.10 - -rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" - -rem Build up classpath -set CLASSPATH=%FWDIR%conf -if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) else ( - for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) -set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% - -if "x%SPARK_TESTING%"=="x1" ( - rem Add test clases to path - set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes -) - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - -rem A bit of a hack to allow calling this script within run2.cmd without seeing output -if "%DONT_PRINT_CLASSPATH%"=="1" goto exit - -echo %CLASSPATH% - -:exit +@echo off + +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 This script computes Spark's classpath and prints it to stdout; it's used by both the "run" +rem script and the ExecutorRunner in standalone cluster mode. + +set SCALA_VERSION=2.10 + +rem Figure out where the Spark framework is installed +set FWDIR=%~dp0..\ + +rem Load environment variables from conf\spark-env.cmd, if it exists +if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" + +rem Build up classpath +set CLASSPATH=%FWDIR%conf +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) else ( + for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) + +set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% + +set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes + +set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes + +if "x%SPARK_TESTING%"=="x1" ( + rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH + rem so that local compilation takes precedence over assembled jar + set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% +) + +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + +rem A bit of a hack to allow calling this script within run2.cmd without seeing output +if "%DONT_PRINT_CLASSPATH%"=="1" goto exit + +echo %CLASSPATH% + +:exit diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 2a2bb376fd71f..b0218531e9eb8 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -28,7 +28,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" . $FWDIR/bin/load-spark-env.sh # Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" +CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" @@ -50,9 +50,9 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then else # Else use spark-assembly jar from either RELEASE or assembly directory if [ -f "$FWDIR/RELEASE" ]; then - ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark*-assembly*.jar` + ASSEMBLY_JAR=`ls "$FWDIR"/lib/spark-assembly*hadoop*.jar` else - ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*.jar` + ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*.jar` fi CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi diff --git a/bin/run-example b/bin/run-example index 5af95a08c6c41..d8a94f2e31e07 100755 --- a/bin/run-example +++ b/bin/run-example @@ -40,12 +40,15 @@ fi # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack # to avoid the -sources and -doc packages that are built by publish-local. EXAMPLES_DIR="$FWDIR"/examples -SPARK_EXAMPLES_JAR="" -if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` + +if [ -f "$FWDIR/RELEASE" ]; then + export SPARK_EXAMPLES_JAR=`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar` +elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar` fi + if [[ -z $SPARK_EXAMPLES_JAR ]]; then - echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 + echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" >&2 echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 exit 1 fi @@ -75,7 +78,6 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$SPARK_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" diff --git a/bin/spark-class b/bin/spark-class index 1b0d309cc5b1c..e8160c8af64c1 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -73,11 +73,13 @@ case "$1" in OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} ;; - # All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS. - 'org.apache.spark.repl.Main') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_REPL_OPTS" + # Spark submit uses SPARK_SUBMIT_OPTS and SPARK_JAVA_OPTS + 'org.apache.spark.deploy.SparkSubmit') + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS \ + -Djava.library.path=$SPARK_SUBMIT_LIBRARY_PATH" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} ;; + *) OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} @@ -98,7 +100,6 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then diff --git a/bin/spark-shell b/bin/spark-shell index ea12d256b23a1..7f03349c5e910 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -19,8 +19,6 @@ # # Shell script for starting the Spark Shell REPL -# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} -# if those two env vars are set in spark-env.sh but MASTER is not. cygwin=false case "`uname`" in @@ -30,133 +28,16 @@ esac # Enter posix mode for bash set -o posix +if [[ "$@" == *--help* ]]; then + echo "Usage: ./bin/spark-shell [options]" + ./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + ## Global script variables FWDIR="$(cd `dirname $0`/..; pwd)" -SPARK_REPL_OPTS="${SPARK_REPL_OPTS:-""}" -DEFAULT_MASTER="local[*]" -MASTER=${MASTER:-""} - -info_log=0 - -#CLI Color Templates -txtund=$(tput sgr 0 1) # Underline -txtbld=$(tput bold) # Bold -bldred=${txtbld}$(tput setaf 1) # red -bldyel=${txtbld}$(tput setaf 3) # yellow -bldblu=${txtbld}$(tput setaf 4) # blue -bldwht=${txtbld}$(tput setaf 7) # white -txtrst=$(tput sgr0) # Reset -info=${bldwht}*${txtrst} # Feedback -pass=${bldblu}*${txtrst} -warn=${bldred}*${txtrst} -ques=${bldblu}?${txtrst} - -# Helper function to describe the script usage -function usage() { - cat << EOF -${txtbld}Usage${txtrst}: spark-shell [OPTIONS] - -${txtbld}OPTIONS${txtrst}: - -h --help : Print this help information. - -c --cores : The maximum number of cores to be used by the Spark Shell. - -em --executor-memory : The memory used by each executor of the Spark Shell, the number - is followed by m for megabytes or g for gigabytes, e.g. "1g". - -dm --driver-memory : The memory used by the Spark Shell, the number is followed - by m for megabytes or g for gigabytes, e.g. "1g". - -m --master : A full string that describes the Spark Master, defaults to "local[*]" - e.g. "spark://localhost:7077". - --log-conf : Enables logging of the supplied SparkConf as INFO at start of the - Spark Context. - -e.g. - spark-shell -m spark://localhost:7077 -c 4 -dm 512m -em 2g - -EOF -} - -function out_error(){ - echo -e "${txtund}${bldred}ERROR${txtrst}: $1" - usage - exit 1 -} - -function log_info(){ - [ $info_log -eq 1 ] && echo -e "${bldyel}INFO${txtrst}: $1" -} - -function log_warn(){ - echo -e "${txtund}${bldyel}WARN${txtrst}: $1" -} - -# PATTERNS used to validate more than one optional arg. -ARG_FLAG_PATTERN="^-" -MEM_PATTERN="^[0-9]+[m|g|M|G]$" -NUM_PATTERN="^[0-9]+$" -PORT_PATTERN="^[0-9]+$" - -# Setters for optional args. -function set_cores(){ - CORE_PATTERN="^[0-9]+$" - if [[ "$1" =~ $CORE_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1" - else - out_error "wrong format for $2" - fi -} - -function set_em(){ - if [[ $1 =~ $MEM_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.executor.memory=$1" - else - out_error "wrong format for $2" - fi -} - -function set_dm(){ - if [[ $1 =~ $MEM_PATTERN ]]; then - export SPARK_DRIVER_MEMORY=$1 - else - out_error "wrong format for $2" - fi -} - -function set_spark_log_conf(){ - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.logConf=$1" -} - -function set_spark_master(){ - if ! [[ "$1" =~ $ARG_FLAG_PATTERN ]]; then - export MASTER="$1" - else - out_error "wrong format for $2" - fi -} - -function resolve_spark_master(){ - # Set MASTER from spark-env if possible - DEFAULT_SPARK_MASTER_PORT=7077 - if [ -z "$MASTER" ]; then - . $FWDIR/bin/load-spark-env.sh - if [ -n "$SPARK_MASTER_IP" ]; then - SPARK_MASTER_PORT="${SPARK_MASTER_PORT:-"$DEFAULT_SPARK_MASTER_PORT"}" - export MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - fi - fi - - if [ -z "$MASTER" ]; then - export MASTER="$DEFAULT_MASTER" - fi - -} - function main(){ - log_info "Base Directory set to $FWDIR" - - resolve_spark_master - log_info "Spark Master is $MASTER" - - log_info "Spark REPL options $SPARK_REPL_OPTS" if $cygwin; then # Workaround for issue involving JLine and Cygwin # (see http://sourceforge.net/p/jline/bugs/40/). @@ -164,56 +45,15 @@ function main(){ # "Backspace sends ^H" setting in "Keys" section of the Mintty options # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 - export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" + $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else - export SPARK_REPL_OPTS - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + export SPARK_SUBMIT_OPTS + $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main fi } -for option in "$@" -do - case $option in - -h | --help ) - usage - exit 1 - ;; - -c | --cores) - shift - _1=$1 - shift - set_cores $_1 "-c/--cores" - ;; - -em | --executor-memory) - shift - _1=$1 - shift - set_em $_1 "-em/--executor-memory" - ;; - -dm | --driver-memory) - shift - _1=$1 - shift - set_dm $_1 "-dm/--driver-memory" - ;; - -m | --master) - shift - _1=$1 - shift - set_spark_master $_1 "-m/--master" - ;; - --log-conf) - shift - set_spark_log_conf "true" - info_log=1 - ;; - ?) - ;; - esac -done - # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed exit_status=127 @@ -242,7 +82,7 @@ if [[ ! $? ]]; then saved_stty="" fi -main +main "$@" # record the exit status lest it be overwritten: # then reenable echo and propagate the code. diff --git a/bin/spark-submit b/bin/spark-submit index d92d55a032bd5..dd0d95d9d4002 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -21,12 +21,17 @@ export SPARK_HOME="$(cd `dirname $0`/..; pwd)" ORIG_ARGS=$@ while (($#)); do - if [ $1 = "--deploy-mode" ]; then + if [ "$1" = "--deploy-mode" ]; then DEPLOY_MODE=$2 - elif [ $1 = "--driver-memory" ]; then + elif [ "$1" = "--driver-memory" ]; then DRIVER_MEMORY=$2 + elif [ "$1" = "--driver-library-path" ]; then + export SPARK_SUBMIT_LIBRARY_PATH=$2 + elif [ "$1" = "--driver-class-path" ]; then + export SPARK_SUBMIT_CLASSPATH=$2 + elif [ "$1" = "--driver-java-options" ]; then + export SPARK_SUBMIT_OPTS=$2 fi - shift done diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template new file mode 100644 index 0000000000000..f840ff681d019 --- /dev/null +++ b/conf/spark-defaults.conf.template @@ -0,0 +1,7 @@ +# Default system properties included when running spark-submit. +# This is useful for setting default environmental settings. + +# Example: +# spark.master spark://master:7077 +# spark.eventLog.enabled true +# spark.eventLog.dir hdfs://namenode:8021/directory diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 6432a566089be..f906be611a931 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -1,22 +1,43 @@ #!/usr/bin/env bash -# This file contains environment variables required to run Spark. Copy it as -# spark-env.sh and edit that to configure Spark for your site. -# -# The following variables can be set in this file: +# This file is sourced when running various Spark programs. +# Copy it as spark-env.sh and edit that to configure Spark for your site. + +# Options read when launching programs locally with +# ./bin/run-example or ./bin/spark-submit +# - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files +# - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# - SPARK_PUBLIC_DNS, to set the public dns name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append + +# Options read by executors and drivers running inside the cluster # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append +# - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos -# - SPARK_JAVA_OPTS, to set node-specific JVM options for Spark. Note that -# we recommend setting app-wide options in the application's driver program. -# Examples of node-specific options : -Dspark.local.dir, GC options -# Examples of app-wide options : -Dspark.serializer -# -# If using the standalone deploy mode, you can also set variables for it here: + +# Options read in YARN client mode +# - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files +# - SPARK_EXECUTOR_INSTANCES, Number of workers to start (Default: 2) +# - SPARK_EXECUTOR_CORES, Number of cores for the workers (Default: 1). +# - SPARK_EXECUTOR_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) +# - SPARK_DRIVER_MEMORY, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) +# - SPARK_YARN_APP_NAME, The name of your application (Default: Spark) +# - SPARK_YARN_QUEUE, The hadoop queue to use for allocation requests (Default: ‘default’) +# - SPARK_YARN_DIST_FILES, Comma separated list of files to be distributed with the job. +# - SPARK_YARN_DIST_ARCHIVES, Comma separated list of archives to be distributed with the job. + +# Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports +# - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine -# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) +# - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes -# - SPARK_PUBLIC_DNS, to set the public dns name of the master +# - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") +# - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") +# - SPARK_DAEMON_OPTS, to set config properties for all daemons (e.g. "-Dx=y") +# - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers diff --git a/core/pom.xml b/core/pom.xml index d87e2bca030e3..73f573a414050 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -30,19 +30,6 @@ jar Spark Project Core http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.hadoop @@ -147,15 +134,6 @@ org.json4s json4s-jackson_${scala.binary.version} 3.2.6 - - - - org.scala-lang - scalap - - colt @@ -269,7 +247,7 @@ org.spark-project pyrolite - 2.0 + 2.0.1 diff --git a/core/src/main/java/org/apache/spark/package-info.java b/core/src/main/java/org/apache/spark/package-info.java new file mode 100644 index 0000000000000..4426c7afcebdd --- /dev/null +++ b/core/src/main/java/org/apache/spark/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ + +/** + * Core Spark classes in Scala. A few classes here, such as {@link org.apache.spark.Accumulator} + * and {@link org.apache.spark.storage.StorageLevel}, are also used in Java, but the + * {@link org.apache.spark.api.java} package contains the main Java API. + */ +package org.apache.spark; diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index d5f3e3f6ec496..6d652faae149a 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -104,8 +104,11 @@ class Accumulable[R, T] ( * Set the accumulator's value; only allowed on master. */ def value_= (newValue: R) { - if (!deserialized) value_ = newValue - else throw new UnsupportedOperationException("Can't assign accumulator value in task") + if (!deserialized) { + value_ = newValue + } else { + throw new UnsupportedOperationException("Can't assign accumulator value in task") + } } /** diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index c7893f288b4b5..811610c657b62 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -47,7 +47,12 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { if (loading.contains(key)) { logInfo("Another thread is loading %s, waiting for it to finish...".format(key)) while (loading.contains(key)) { - try {loading.wait()} catch {case _ : Throwable =>} + try { + loading.wait() + } catch { + case e: Exception => + logWarning(s"Got an exception while waiting for another thread to load $key", e) + } } logInfo("Finished waiting for %s".format(key)) /* See whether someone else has successfully loaded it. The main way this would fail @@ -72,7 +77,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val computedValues = rdd.computeOrReadCheckpoint(split, context) // Persist the result, so long as the task is not running locally - if (context.runningLocally) { return computedValues } + if (context.runningLocally) { + return computedValues + } // Keep track of blocks with updated statuses var updatedBlocks = Seq[(BlockId, BlockStatus)]() @@ -88,7 +95,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { updatedBlocks = blockManager.put(key, computedValues, storageLevel, tellMaster = true) blockManager.get(key) match { case Some(values) => - new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) + values.asInstanceOf[Iterator[T]] case None => logInfo("Failure to store %s".format(key)) throw new Exception("Block manager failed to return persisted valued") @@ -107,7 +114,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val metrics = context.taskMetrics metrics.updatedBlocks = Some(updatedBlocks) - returnValue + new InterruptibleIterator(context, returnValue) } finally { loading.synchronized { diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index fd1802ba2f984..ec11dbbffaaf8 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -24,7 +24,17 @@ package org.apache.spark private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { - def hasNext: Boolean = !context.interrupted && delegate.hasNext + def hasNext: Boolean = { + // TODO(aarondav/rxin): Check Thread.interrupted instead of context.interrupted if interrupt + // is allowed. The assumption is that Thread.interrupted does not have a memory fence in read + // (just a volatile field in C), while context.interrupted is a volatile in the JVM, which + // introduces an expensive read fence. + if (context.interrupted) { + throw new TaskKilledException + } else { + delegate.hasNext + } + } def next(): T = delegate.next() } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index b52f2d4f416b2..b4b0067801259 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -19,8 +19,6 @@ package org.apache.spark import java.net.{Authenticator, PasswordAuthentication} -import scala.collection.mutable.ArrayBuffer - import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil @@ -139,13 +137,13 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { private val sparkSecretLookupKey = "sparkCookie" private val authOn = sparkConf.getBoolean("spark.authenticate", false) - private val uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + private var uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + private var viewAcls: Set[String] = _ // always add the current user and SPARK_USER to the viewAcls - private val aclUsers = ArrayBuffer[String](System.getProperty("user.name", ""), + private val defaultAclUsers = Seq[String](System.getProperty("user.name", ""), Option(System.getenv("SPARK_USER")).getOrElse("")) - aclUsers ++= sparkConf.get("spark.ui.view.acls", "").split(',') - private val viewAcls = aclUsers.map(_.trim()).filter(!_.isEmpty).toSet + setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) private val secretKey = generateSecretKey() logInfo("SecurityManager, is authentication enabled: " + authOn + @@ -170,6 +168,20 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { ) } + private[spark] def setViewAcls(defaultUsers: Seq[String], allowedUsers: String) { + viewAcls = (defaultUsers ++ allowedUsers.split(',')).map(_.trim()).filter(!_.isEmpty).toSet + logInfo("Changing view acls to: " + viewAcls.mkString(",")) + } + + private[spark] def setViewAcls(defaultUser: String, allowedUsers: String) { + setViewAcls(Seq[String](defaultUser), allowedUsers) + } + + private[spark] def setUIAcls(aclSetting: Boolean) { + uiAclsOn = aclSetting + logInfo("Changing acls enabled to: " + uiAclsOn) + } + /** * Generates or looks up the secret key. * @@ -222,6 +234,8 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { + logDebug("user=" + user + " uiAclsEnabled=" + uiAclsEnabled() + " viewAcls=" + + viewAcls.mkString(",")) if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true } diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b947feb891ee6..bd21fdc5a18e4 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -208,6 +208,82 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { new SparkConf(false).setAll(settings) } + /** Checks for illegal or deprecated config settings. Throws an exception for the former. Not + * idempotent - may mutate this conf object to convert deprecated settings to supported ones. */ + private[spark] def validateSettings() { + if (settings.contains("spark.local.dir")) { + val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " + + "the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone and LOCAL_DIRS in YARN)." + logWarning(msg) + } + + val executorOptsKey = "spark.executor.extraJavaOptions" + val executorClasspathKey = "spark.executor.extraClassPath" + val driverOptsKey = "spark.driver.extraJavaOptions" + val driverClassPathKey = "spark.driver.extraClassPath" + + // Validate spark.executor.extraJavaOptions + settings.get(executorOptsKey).map { javaOpts => + if (javaOpts.contains("-Dspark")) { + val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts)'. " + + "Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit." + throw new Exception(msg) + } + if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { + val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). " + + "Use spark.executor.memory instead." + throw new Exception(msg) + } + } + + // Check for legacy configs + sys.env.get("SPARK_JAVA_OPTS").foreach { value => + val error = + s""" + |SPARK_JAVA_OPTS was detected (set to '$value'). + |This has undefined behavior when running on a cluster and 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 + | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) + """.stripMargin + logError(error) + + for (key <- Seq(executorOptsKey, driverOptsKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } + + sys.env.get("SPARK_CLASSPATH").foreach { value => + val error = + s""" + |SPARK_CLASSPATH was detected (set to '$value'). + | This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with --driver-class-path to augment the driver classpath + | - spark.executor.extraClassPath to augment the executor classpath + """.stripMargin + logError(error) + + for (key <- Seq(executorClasspathKey, driverClassPathKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_CLASSPATH. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } + } + /** * Return a string listing all keys and values, one per line. This is useful to print the * configuration out for debugging. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d3ef75bc7335a..eb14d87467af7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -148,6 +148,7 @@ class SparkContext(config: SparkConf) extends Logging { this(master, appName, sparkHome, jars, Map(), Map()) private[spark] val conf = config.clone() + conf.validateSettings() /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be @@ -159,7 +160,7 @@ class SparkContext(config: SparkConf) extends Logging { throw new SparkException("A master URL must be set in your configuration") } if (!conf.contains("spark.app.name")) { - throw new SparkException("An application must be set in your configuration") + throw new SparkException("An application name must be set in your configuration") } if (conf.getBoolean("spark.logConf", false)) { @@ -170,11 +171,11 @@ class SparkContext(config: SparkConf) extends Logging { conf.setIfMissing("spark.driver.host", Utils.localHostName()) conf.setIfMissing("spark.driver.port", "0") - val jars: Seq[String] = if (conf.contains("spark.jars")) { - conf.get("spark.jars").split(",").filter(_.size != 0) - } else { - null - } + val jars: Seq[String] = + conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten + + val files: Seq[String] = + conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten val master = conf.get("spark.master") val appName = conf.get("spark.app.name") @@ -215,10 +216,33 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val ui = new SparkUI(this) ui.bind() + /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ + val hadoopConfiguration: Configuration = { + val env = SparkEnv.get + val hadoopConf = SparkHadoopUtil.get.newConfiguration() + // Explicitly check for S3 environment variables + if (System.getenv("AWS_ACCESS_KEY_ID") != null && + System.getenv("AWS_SECRET_ACCESS_KEY") != null) { + hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + } + // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" + conf.getAll.foreach { case (key, value) => + if (key.startsWith("spark.hadoop.")) { + hadoopConf.set(key.substring("spark.hadoop.".length), value) + } + } + val bufferSize = conf.get("spark.buffer.size", "65536") + hadoopConf.set("io.file.buffer.size", bufferSize) + hadoopConf + } + // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { if (conf.getBoolean("spark.eventLog.enabled", false)) { - val logger = new EventLoggingListener(appName, conf) + val logger = new EventLoggingListener(appName, conf, hadoopConfiguration) logger.start() listenerBus.addListener(logger) Some(logger) @@ -235,6 +259,10 @@ class SparkContext(config: SparkConf) extends Logging { jars.foreach(addJar) } + if (files != null) { + files.foreach(addFile) + } + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") @@ -247,22 +275,20 @@ class SparkContext(config: SparkConf) extends Logging { .map(Utils.memoryStringToMb) .getOrElse(512) - // Environment variables to pass to our executors - private[spark] val executorEnvs = HashMap[String, String]() - for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS"); - value <- Option(System.getenv(key))) { - executorEnvs(key) = value - } + // Environment variables to pass to our executors. + // NOTE: This should only be used for test related settings. + private[spark] val testExecutorEnvs = HashMap[String, String]() + // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) + for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - executorEnvs(envKey) = value + testExecutorEnvs(envKey) = value } // The Mesos scheduler backend relies on this environment variable to set executor memory. // TODO: Set this only in the Mesos scheduler. - executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - executorEnvs ++= conf.getExecutorEnv + testExecutorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + testExecutorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { @@ -270,14 +296,21 @@ class SparkContext(config: SparkConf) extends Logging { }.getOrElse { SparkContext.SPARK_UNKNOWN_USER } - executorEnvs("SPARK_USER") = sparkUser + testExecutorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) - taskScheduler.start() + @volatile private[spark] var dagScheduler: DAGScheduler = _ + try { + dagScheduler = new DAGScheduler(this) + } catch { + case e: Exception => throw + new SparkException("DAGScheduler cannot be initialized due to %s".format(e.getMessage)) + } - @volatile private[spark] var dagScheduler = new DAGScheduler(this) - dagScheduler.start() + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's + // constructor + taskScheduler.start() private[spark] val cleaner: Option[ContextCleaner] = { if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { @@ -291,29 +324,6 @@ class SparkContext(config: SparkConf) extends Logging { postEnvironmentUpdate() postApplicationStart() - /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ - val hadoopConfiguration: Configuration = { - val env = SparkEnv.get - val hadoopConf = SparkHadoopUtil.get.newConfiguration() - // Explicitly check for S3 environment variables - if (System.getenv("AWS_ACCESS_KEY_ID") != null && - System.getenv("AWS_SECRET_ACCESS_KEY") != null) { - hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - } - // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" - conf.getAll.foreach { case (key, value) => - if (key.startsWith("spark.hadoop.")) { - hadoopConf.set(key.substring("spark.hadoop.".length), value) - } - } - val bufferSize = conf.get("spark.buffer.size", "65536") - hadoopConf.set("io.file.buffer.size", bufferSize) - hadoopConf - } - private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack @@ -378,16 +388,27 @@ class SparkContext(config: SparkConf) extends Logging { * // In a separate thread: * sc.cancelJobGroup("some_job_to_cancel") * }}} + * + * If interruptOnCancel is set to true for the job group, then job cancellation will result + * in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + * that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + * where HDFS may respond to Thread.interrupt() by marking nodes as dead. */ - def setJobGroup(groupId: String, description: String) { + def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean = false) { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, description) setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId) + // Note: Specifying interruptOnCancel in setJobGroup (rather than cancelJobGroup) avoids + // changing several public APIs and allows Spark cancellations outside of the cancelJobGroup + // APIs to also take advantage of this property (e.g., internal job failures or canceling from + // JobProgressTab UI) on a per-job basis. + setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, interruptOnCancel.toString) } /** Clear the current thread's job group ID and its description. */ def clearJobGroup() { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, null) setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, null) + setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) } // Post init @@ -1008,8 +1029,8 @@ class SparkContext(config: SparkConf) extends Logging { partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { - partitions.foreach{ p => - require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") + if (dagScheduler == null) { + throw new SparkException("SparkContext has been shutdown") } val callSite = getCallSite val cleanedFunc = clean(func) @@ -1107,6 +1128,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** + * :: Experimental :: * Submit a job for execution and return a FutureJob holding the result. */ @Experimental @@ -1117,9 +1139,6 @@ class SparkContext(config: SparkConf) extends Logging { resultHandler: (Int, U) => Unit, resultFunc: => R): SimpleFutureAction[R] = { - partitions.foreach{ p => - require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") - } val cleanF = clean(processPartition) val callSite = getCallSite val waiter = dagScheduler.submitJob( @@ -1240,6 +1259,8 @@ object SparkContext extends Logging { private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" + private[spark] val SPARK_JOB_INTERRUPT_ON_CANCEL = "spark.job.interruptOnCancel" + private[spark] val SPARK_UNKNOWN_USER = "" implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { @@ -1264,8 +1285,10 @@ object SparkContext extends Logging { // TODO: Add AccumulatorParams for other types, e.g. lists and strings - implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { new PairRDDFunctions(rdd) + } implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) @@ -1342,19 +1365,19 @@ object SparkContext extends Logging { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to SparkContext. */ - def jarOfClass(cls: Class[_]): Seq[String] = { + def jarOfClass(cls: Class[_]): Option[String] = { val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class") if (uri != null) { val uriStr = uri.toString if (uriStr.startsWith("jar:file:")) { // URI will be of the form "jar:file:/path/foo.jar!/package/cls.class", // so pull out the /path/foo.jar - List(uriStr.substring("jar:file:".length, uriStr.indexOf('!'))) + Some(uriStr.substring("jar:file:".length, uriStr.indexOf('!'))) } else { - Nil + None } } else { - Nil + None } } @@ -1363,7 +1386,7 @@ object SparkContext extends Logging { * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in * your driver program. */ - def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) + def jarOfObject(obj: AnyRef): Option[String] = jarOfClass(obj.getClass) /** * Creates a modified version of a SparkConf with the parameters that can be passed separately diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 915315ed74436..bea435ec34ce9 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -97,6 +97,14 @@ class SparkEnv ( pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } + + private[spark] + def destroyPythonWorker(pythonExec: String, envVars: Map[String, String]) { + synchronized { + val key = (pythonExec, envVars) + pythonWorkers(key).stop() + } + } } object SparkEnv extends Logging { diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala index 5b14c4291d91a..65003b6ac6a0a 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -111,10 +111,10 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) private val secretKey = securityMgr.getSecretKey() - private val userPassword: Array[Char] = - SparkSaslServer.encodePassword(if (secretKey != null) secretKey.getBytes() else "".getBytes()) + private val userPassword: Array[Char] = SparkSaslServer.encodePassword( + if (secretKey != null) secretKey.getBytes("utf-8") else "".getBytes("utf-8")) /** * Implementation used to respond to SASL request from the server. diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala index 6161a6fb7ae85..f6b0a9132aca4 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -89,7 +89,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi extends CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) override def handle(callbacks: Array[Callback]) { logDebug("In the sasl server callback handler") @@ -101,7 +101,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi case pc: PasswordCallback => { logDebug("handle: SASL server callback: setting userPassword") val password: Array[Char] = - SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes()) + SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes("utf-8")) pc.setPassword(password) } case rc: RealmCallback => { @@ -159,7 +159,7 @@ private[spark] object SparkSaslServer { * @return Base64-encoded string */ def encodeIdentifier(identifier: Array[Byte]): String = { - new String(Base64.encodeBase64(identifier)) + new String(Base64.encodeBase64(identifier), "utf-8") } /** @@ -168,7 +168,7 @@ private[spark] object SparkSaslServer { * @return password as a char array. */ def encodePassword(password: Array[Byte]): Array[Char] = { - new String(Base64.encodeBase64(password)).toCharArray() + new String(Base64.encodeBase64(password), "utf-8").toCharArray() } } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index dc5a19ecd738e..dc012cc381346 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -33,7 +33,7 @@ class TaskContext( val attemptId: Long, val runningLocally: Boolean = false, @volatile var interrupted: Boolean = false, - private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty() + private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty ) extends Serializable { @deprecated("use partitionId", "0.8.1") diff --git a/core/src/main/scala/org/apache/spark/TaskKilledException.scala b/core/src/main/scala/org/apache/spark/TaskKilledException.scala new file mode 100644 index 0000000000000..cbd6b2866e4f9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskKilledException.scala @@ -0,0 +1,23 @@ +/* + * 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 + +/** + * Exception for a task getting killed. + */ +private[spark] class TaskKilledException extends RuntimeException diff --git a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java index af01fb7cfbd04..db7b25c727d34 100644 --- a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java +++ b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java @@ -19,7 +19,14 @@ import java.lang.annotation.*; -/** A new component of Spark which may have unstable API's. */ +/** + * A new component of Spark which may have unstable API's. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: AlphaComponent ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. + */ @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) diff --git a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java index 5d546e7a63985..0ecef6db0e039 100644 --- a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java +++ b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java @@ -23,6 +23,11 @@ * A lower-level, unstable API intended for developers. * * Developer API's might change or be removed in minor versions of Spark. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: DeveloperApi ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. */ @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, diff --git a/core/src/main/scala/org/apache/spark/annotation/Experimental.java b/core/src/main/scala/org/apache/spark/annotation/Experimental.java index 306b1418d8d0a..ff8120291455f 100644 --- a/core/src/main/scala/org/apache/spark/annotation/Experimental.java +++ b/core/src/main/scala/org/apache/spark/annotation/Experimental.java @@ -24,6 +24,11 @@ * * Experimental API's might change or be removed in minor versions of Spark, or be adopted as * first-class Spark API's. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: Experimental ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. */ @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 4330cef3965ee..a6123bd108c11 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -30,6 +30,7 @@ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.StatCounter +import org.apache.spark.util.Utils class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, JavaDoubleRDD] { @@ -133,7 +134,13 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: JDouble, seed: Int): JavaDoubleRDD = + def sample(withReplacement: Boolean, fraction: JDouble): JavaDoubleRDD = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: JDouble, seed: Long): JavaDoubleRDD = fromRDD(srdd.sample(withReplacement, fraction, seed)) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index b3ec270281ae4..554c065358648 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -39,6 +39,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) (implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V]) @@ -119,7 +120,13 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaPairRDD[K, V] = + def sample(withReplacement: Boolean, fraction: Double): JavaPairRDD[K, V] = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sample(withReplacement, fraction, seed)) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 327c1552dc941..dc698dea75e43 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -24,6 +24,7 @@ import org.apache.spark._ import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) extends JavaRDDLike[T, JavaRDD[T]] { @@ -98,7 +99,13 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] = + def sample(withReplacement: Boolean, fraction: Double): JavaRDD[T] = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 725c423a53e35..574a98636a619 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -34,6 +34,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def wrapRDD(rdd: RDD[T]): This @@ -394,7 +395,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } - def takeSample(withReplacement: Boolean, num: Int, seed: Int): JList[T] = { + def takeSample(withReplacement: Boolean, num: Int): JList[T] = + takeSample(withReplacement, num, Utils.random.nextLong) + + def takeSample(withReplacement: Boolean, num: Int, seed: Long): JList[T] = { import scala.collection.JavaConversions._ val arr: java.util.Collection[T] = rdd.takeSample(withReplacement, num, seed).toSeq new java.util.ArrayList(arr) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index bda9272b43393..8b95cda511643 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -570,6 +570,21 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * // In a separate thread: * sc.cancelJobGroup("some_job_to_cancel"); * }}} + * + * If interruptOnCancel is set to true for the job group, then job cancellation will result + * in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + * that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + * where HDFS may respond to Thread.interrupt() by marking nodes as dead. + */ + def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean): Unit = + sc.setJobGroup(groupId, description, interruptOnCancel) + + /** + * Assigns a group ID to all the jobs started by this thread until the group ID is set to a + * different value or cleared. + * + * @see `setJobGroup(groupId: String, description: String, interruptThread: Boolean)`. + * This method sets interruptOnCancel to false. */ def setJobGroup(groupId: String, description: String): Unit = sc.setJobGroup(groupId, description) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index f9d86fed34d0f..672c344a56597 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,6 +24,7 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import scala.util.Try import net.razorvine.pickle.{Pickler, Unpickler} @@ -89,16 +90,47 @@ private[spark] class PythonRDD[T: ClassTag]( dataOut.flush() worker.shutdownOutput() } catch { + case e: java.io.FileNotFoundException => readerException = e - // Kill the Python worker process: - worker.shutdownOutput() + Try(worker.shutdownOutput()) // kill Python worker process + case e: IOException => // This can happen for legitimate reasons if the Python code stops returning data - // before we are done passing elements through, e.g., for take(). Just log a message - // to say it happened. - logInfo("stdin writer to Python finished early") - logDebug("stdin writer to Python finished early", e) + // before we are done passing elements through, e.g., for take(). Just log a message to + // say it happened (as it could also be hiding a real IOException from a data source). + logInfo("stdin writer to Python finished early (may not be an error)", e) + + case e: Exception => + // We must avoid throwing exceptions here, because the thread uncaught exception handler + // will kill the whole executor (see Executor). + readerException = e + Try(worker.shutdownOutput()) // kill Python worker process + } + } + }.start() + + // Necessary to distinguish between a task that has failed and a task that is finished + @volatile var complete: Boolean = false + + // It is necessary to have a monitor thread for python workers if the user cancels with + // interrupts disabled. In that case we will need to explicitly kill the worker, otherwise the + // threads can block indefinitely. + new Thread(s"Worker Monitor for $pythonExec") { + override def run() { + // Kill the worker if it is interrupted or completed + // When a python task completes, the context is always set to interupted + while (!context.interrupted) { + Thread.sleep(2000) + } + if (!complete) { + try { + logWarning("Incomplete task interrupted: Attempting to kill Python Worker") + env.destroyPythonWorker(pythonExec, envVars.toMap) + } catch { + case e: Exception => + logError("Exception when trying to kill worker", e) + } } } }.start() @@ -109,7 +141,10 @@ private[spark] class PythonRDD[T: ClassTag]( * is not synchronous this still leaves a potential race where the interruption is * processed only after the stream becomes invalid. */ - context.addOnCompleteCallback(() => context.interrupted = true) + context.addOnCompleteCallback{ () => + complete = true // Indicate that the task has completed successfully + context.interrupted = true + } // Return an iterator that read lines from the process's stdout val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) @@ -152,7 +187,7 @@ private[spark] class PythonRDD[T: ClassTag]( val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj)) + throw new PythonException(new String(obj, "utf-8"), readerException) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still // read some accumulator updates: @@ -162,15 +197,17 @@ private[spark] class PythonRDD[T: ClassTag]( val update = new Array[Byte](updateLen) stream.readFully(update) accumulator += Collections.singletonList(update) - } Array.empty[Byte] } } catch { - case eof: EOFException => { + case e: Exception if readerException != null => + logError("Python worker exited unexpectedly (crashed)", e) + logError("Python crash may have been caused by prior exception:", readerException) + throw readerException + + case eof: EOFException => throw new SparkException("Python worker exited unexpectedly (crashed)", eof) - } - case e: Throwable => throw e } } @@ -185,7 +222,7 @@ private[spark] class PythonRDD[T: ClassTag]( } /** Thrown for exceptions in user Python code. */ -private class PythonException(msg: String) extends Exception(msg) +private class PythonException(msg: String, cause: Exception) extends RuntimeException(msg, cause) /** * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python. 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 8fd2c7e95b966..7ead1171525d2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -54,8 +54,21 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends System.getenv().foreach{case (k, v) => env(k) = v} val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" + + val classPathConf = "spark.driver.extraClassPath" + val classPathEntries = sys.props.get(classPathConf).toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + + val libraryPathConf = "spark.driver.extraLibraryPath" + val libraryPathEntries = sys.props.get(libraryPathConf).toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + + val javaOptionsConf = "spark.driver.extraJavaOptions" + val javaOpts = sys.props.get(javaOptionsConf) val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ - driverArgs.driverOptions, env) + driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts) val driverDescription = new DriverDescription( driverArgs.jarUrl, diff --git a/core/src/main/scala/org/apache/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala index fa8af9a646750..32f3ba385084f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Command.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala @@ -22,5 +22,8 @@ import scala.collection.Map private[spark] case class Command( mainClass: String, arguments: Seq[String], - environment: Map[String, String]) { + environment: Map[String, String], + classPathEntries: Seq[String], + libraryPathEntries: Seq[String], + extraJavaOptions: Option[String] = None) { } 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 9bdbfb33bf54f..498fcc520ac5e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -75,6 +75,10 @@ class SparkHadoopUtil { def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { null } + def loginUserFromKeytab(principalName: String, keytabFilename: String) { + UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) + } + } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index e5d593cade8b3..d131f1809c99e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,14 +17,13 @@ package org.apache.spark.deploy -import java.io.{PrintStream, File} +import java.io.{File, PrintStream} import java.net.{URI, URL} -import org.apache.spark.executor.ExecutorURLClassLoader +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map +import org.apache.spark.executor.ExecutorURLClassLoader +import org.apache.spark.util.Utils /** * Scala code behind the spark-submit script. The script handles setting up the classpath with @@ -40,6 +39,12 @@ object SparkSubmit { private var clusterManager: Int = LOCAL + /** + * A special jar name that indicates the class being run is inside of Spark itself, + * and therefore no user jar is needed. + */ + private val RESERVED_JAR_NAME = "spark-internal" + def main(args: Array[String]) { val appArgs = new SparkSubmitArguments(args) if (appArgs.verbose) { @@ -63,7 +68,8 @@ object SparkSubmit { /** * @return * a tuple containing the arguments for the child, a list of classpath - * entries for the child, and the main class for the child + * entries for the child, a list of system propertes, a list of env vars + * and the main class for the child */ private[spark] def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { @@ -114,15 +120,35 @@ object SparkSubmit { if (!deployOnCluster) { childMainClass = appArgs.mainClass - childClasspath += appArgs.primaryResource + if (appArgs.primaryResource != RESERVED_JAR_NAME) { + childClasspath += appArgs.primaryResource + } } else if (clusterManager == YARN) { childMainClass = "org.apache.spark.deploy.yarn.Client" childArgs += ("--jar", appArgs.primaryResource) childArgs += ("--class", appArgs.mainClass) } + if (clusterManager == YARN) { + // The choice of class is arbitrary, could use any spark-yarn class + if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) { + val msg = "Could not load YARN classes. This copy of Spark may not have been compiled " + + "with YARN support." + throw new Exception(msg) + } + } + + // Special flag to avoid deprecation warnings at the client + sysProps("SPARK_SUBMIT") = "true" + val options = List[OptionAssigner]( new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, + sysProp = "spark.driver.extraClassPath"), + new OptionAssigner(appArgs.driverExtraJavaOptions, STANDALONE | YARN, true, + sysProp = "spark.driver.extraJavaOptions"), + new OptionAssigner(appArgs.driverExtraLibraryPath, STANDALONE | YARN, true, + sysProp = "spark.driver.extraLibraryPath"), new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), @@ -142,10 +168,14 @@ object SparkSubmit { new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), - new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars") + new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars"), + new OptionAssigner(appArgs.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), + new OptionAssigner(appArgs.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars"), + new OptionAssigner(appArgs.name, LOCAL | STANDALONE | MESOS, false, + sysProp = "spark.app.name") ) - // more jars + // For client mode make any added jars immediately visible on the classpath if (appArgs.jars != null && !deployOnCluster) { for (jar <- appArgs.jars.split(",")) { childClasspath += jar @@ -163,6 +193,13 @@ object SparkSubmit { } } + // For standalone mode, add the application jar automatically so the user doesn't have to + // call sc.addJar. TODO: Standalone mode in the cluster + if (clusterManager == STANDALONE) { + val existingJars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) + sysProps.put("spark.jars", (existingJars ++ Seq(appArgs.primaryResource)).mkString(",")) + } + if (deployOnCluster && clusterManager == STANDALONE) { if (appArgs.supervise) { childArgs += "--supervise" @@ -173,17 +210,21 @@ object SparkSubmit { childArgs += (appArgs.master, appArgs.primaryResource, appArgs.mainClass) } - // args + // Arguments to be passed to user program if (appArgs.childArgs != null) { if (!deployOnCluster || clusterManager == STANDALONE) { childArgs ++= appArgs.childArgs } else if (clusterManager == YARN) { for (arg <- appArgs.childArgs) { - childArgs += ("--args", arg) + childArgs += ("--arg", arg) } } } + for ((k, v) <- appArgs.getDefaultSparkProperties) { + if (!sysProps.contains(k)) sysProps(k) = v + } + (childArgs, childClasspath, sysProps, childMainClass) } @@ -191,11 +232,11 @@ object SparkSubmit { sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) { if (verbose) { - System.err.println(s"Main class:\n$childMainClass") - System.err.println(s"Arguments:\n${childArgs.mkString("\n")}") - System.err.println(s"System properties:\n${sysProps.mkString("\n")}") - System.err.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") - System.err.println("\n") + printStream.println(s"Main class:\n$childMainClass") + printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") + printStream.println(s"System properties:\n${sysProps.mkString("\n")}") + printStream.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") + printStream.println("\n") } val loader = new ExecutorURLClassLoader(new Array[URL](0), @@ -226,6 +267,10 @@ object SparkSubmit { } } +/** + * Provides an indirection layer for passing arguments as system properties or flags to + * the user's driver program or to downstream launcher tools. + */ private[spark] class OptionAssigner(val value: String, val clusterManager: Int, val deployOnCluster: Boolean, 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 834b3df2f164b..5834dc40f1f95 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -17,18 +17,29 @@ package org.apache.spark.deploy -import scala.collection.mutable.ArrayBuffer +import java.io.{File, FileInputStream, IOException} +import java.util.Properties + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, HashMap} + +import org.apache.spark.SparkException +import org.apache.spark.util.Utils /** * Parses and encapsulates arguments from the spark-submit script. */ -private[spark] class SparkSubmitArguments(args: Array[String]) { - var master: String = "local" +private[spark] class SparkSubmitArguments(args: Seq[String]) { + var master: String = null var deployMode: String = null var executorMemory: String = null var executorCores: String = null var totalExecutorCores: String = null + var propertiesFile: String = null var driverMemory: String = null + var driverExtraClassPath: String = null + var driverExtraLibraryPath: String = null + var driverExtraJavaOptions: String = null var driverCores: String = null var supervise: Boolean = false var queue: String = null @@ -42,127 +53,224 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { var jars: String = null var verbose: Boolean = false - loadEnvVars() parseOpts(args.toList) + loadDefaults() + checkRequiredArguments() + + /** Return default present in the currently defined defaults file. */ + def getDefaultSparkProperties = { + val defaultProperties = new HashMap[String, String]() + if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") + Option(propertiesFile).foreach { filename => + val file = new File(filename) + SparkSubmitArguments.getPropertiesFromFile(file).foreach { case (k, v) => + if (k.startsWith("spark")) { + defaultProperties(k) = v + if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") + } else { + SparkSubmit.printWarning(s"Ignoring non-spark config property: $k=$v") + } + } + } + defaultProperties + } + + /** Fill in any undefined values based on the current properties file or built-in defaults. */ + private def loadDefaults() = { + + // Use common defaults file, if not specified by user + if (propertiesFile == null) { + sys.env.get("SPARK_HOME").foreach { sparkHome => + val sep = File.separator + val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.conf" + val file = new File(defaultPath) + if (file.exists()) { + propertiesFile = file.getAbsolutePath + } + } + } - // Sanity checks - if (args.length == 0) printUsageAndExit(-1) - if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") - if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + val defaultProperties = getDefaultSparkProperties + // Use properties file as fallback for values which have a direct analog to + // arguments in this script. + master = Option(master).getOrElse(defaultProperties.get("spark.master").orNull) + executorMemory = Option(executorMemory) + .getOrElse(defaultProperties.get("spark.executor.memory").orNull) + executorCores = Option(executorCores) + .getOrElse(defaultProperties.get("spark.executor.cores").orNull) + totalExecutorCores = Option(totalExecutorCores) + .getOrElse(defaultProperties.get("spark.cores.max").orNull) + name = Option(name).getOrElse(defaultProperties.get("spark.app.name").orNull) + jars = Option(jars).getOrElse(defaultProperties.get("spark.jars").orNull) + + // This supports env vars in older versions of Spark + master = Option(master).getOrElse(System.getenv("MASTER")) + deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + + // Global defaults. These should be keep to minimum to avoid confusing behavior. + master = Option(master).getOrElse("local[*]") + } + + /** Ensure that required fields exists. Call this only once all defaults are loaded. */ + private def checkRequiredArguments() = { + if (args.length == 0) printUsageAndExit(-1) + if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") + if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + + if (master.startsWith("yarn")) { + val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") + if (!hasHadoopEnv && !Utils.isTesting) { + throw new Exception(s"When running with master '$master' " + + "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.") + } + } + } override def toString = { s"""Parsed arguments: - | master $master - | deployMode $deployMode - | executorMemory $executorMemory - | executorCores $executorCores - | totalExecutorCores $totalExecutorCores - | driverMemory $driverMemory - | drivercores $driverCores - | supervise $supervise - | queue $queue - | numExecutors $numExecutors - | files $files - | archives $archives - | mainClass $mainClass - | primaryResource $primaryResource - | name $name - | childArgs [${childArgs.mkString(" ")}] - | jars $jars - | verbose $verbose + | master $master + | deployMode $deployMode + | executorMemory $executorMemory + | executorCores $executorCores + | totalExecutorCores $totalExecutorCores + | propertiesFile $propertiesFile + | driverMemory $driverMemory + | driverCores $driverCores + | driverExtraClassPath $driverExtraClassPath + | driverExtraLibraryPath $driverExtraLibraryPath + | driverExtraJavaOptions $driverExtraJavaOptions + | supervise $supervise + | queue $queue + | numExecutors $numExecutors + | files $files + | archives $archives + | mainClass $mainClass + | primaryResource $primaryResource + | name $name + | childArgs [${childArgs.mkString(" ")}] + | jars $jars + | verbose $verbose + | + |Default properties from $propertiesFile: + |${getDefaultSparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } - private def loadEnvVars() { - Option(System.getenv("MASTER")).map(master = _) - Option(System.getenv("DEPLOY_MODE")).map(deployMode = _) - } + /** Fill in values by parsing user options. */ + private def parseOpts(opts: Seq[String]): Unit = { + // Delineates parsing of Spark options from parsing of user options. + var inSparkOpts = true + parse(opts) - private def parseOpts(opts: List[String]): Unit = opts match { - case ("--name") :: value :: tail => - name = value - parseOpts(tail) + def parse(opts: Seq[String]): Unit = opts match { + case ("--name") :: value :: tail => + name = value + parse(tail) - case ("--master") :: value :: tail => - master = value - parseOpts(tail) + case ("--master") :: value :: tail => + master = value + parse(tail) - case ("--class") :: value :: tail => - mainClass = value - parseOpts(tail) + case ("--class") :: value :: tail => + mainClass = value + parse(tail) - case ("--deploy-mode") :: value :: tail => - if (value != "client" && value != "cluster") { - SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") - } - deployMode = value - parseOpts(tail) + case ("--deploy-mode") :: value :: tail => + if (value != "client" && value != "cluster") { + SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") + } + deployMode = value + parse(tail) - case ("--num-executors") :: value :: tail => - numExecutors = value - parseOpts(tail) + case ("--num-executors") :: value :: tail => + numExecutors = value + parse(tail) - case ("--total-executor-cores") :: value :: tail => - totalExecutorCores = value - parseOpts(tail) + case ("--total-executor-cores") :: value :: tail => + totalExecutorCores = value + parse(tail) - case ("--executor-cores") :: value :: tail => - executorCores = value - parseOpts(tail) + case ("--executor-cores") :: value :: tail => + executorCores = value + parse(tail) - case ("--executor-memory") :: value :: tail => - executorMemory = value - parseOpts(tail) + case ("--executor-memory") :: value :: tail => + executorMemory = value + parse(tail) - case ("--driver-memory") :: value :: tail => - driverMemory = value - parseOpts(tail) + case ("--driver-memory") :: value :: tail => + driverMemory = value + parse(tail) - case ("--driver-cores") :: value :: tail => - driverCores = value - parseOpts(tail) + case ("--driver-cores") :: value :: tail => + driverCores = value + parse(tail) - case ("--supervise") :: tail => - supervise = true - parseOpts(tail) + case ("--driver-class-path") :: value :: tail => + driverExtraClassPath = value + parse(tail) - case ("--queue") :: value :: tail => - queue = value - parseOpts(tail) + case ("--driver-java-options") :: value :: tail => + driverExtraJavaOptions = value + parse(tail) - case ("--files") :: value :: tail => - files = value - parseOpts(tail) + case ("--driver-library-path") :: value :: tail => + driverExtraLibraryPath = value + parse(tail) - case ("--archives") :: value :: tail => - archives = value - parseOpts(tail) + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) - case ("--arg") :: value :: tail => - childArgs += value - parseOpts(tail) + case ("--supervise") :: tail => + supervise = true + parse(tail) - case ("--jars") :: value :: tail => - jars = value - parseOpts(tail) + case ("--queue") :: value :: tail => + queue = value + parse(tail) - case ("--help" | "-h") :: tail => - printUsageAndExit(0) + case ("--files") :: value :: tail => + files = value + parse(tail) - case ("--verbose" | "-v") :: tail => - verbose = true - parseOpts(tail) + case ("--archives") :: value :: tail => + archives = value + parse(tail) - case value :: tail => - if (primaryResource != null) { - val error = s"Found two conflicting resources, $value and $primaryResource." + - " Expecting only one resource." - SparkSubmit.printErrorAndExit(error) - } - primaryResource = value - parseOpts(tail) + case ("--jars") :: value :: tail => + jars = value + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case ("--verbose" | "-v") :: tail => + verbose = true + parse(tail) - case Nil => + case value :: tail => + if (inSparkOpts) { + value match { + // convert --foo=bar to --foo bar + case v if v.startsWith("--") && v.contains("=") && v.split("=").size == 2 => + val parts = v.split("=") + parse(Seq(parts(0), parts(1)) ++ tail) + case v if v.startsWith("-") => + val errMessage = s"Unrecognized option '$value'." + SparkSubmit.printErrorAndExit(errMessage) + case v => + primaryResource = v + inSparkOpts = false + parse(tail) + } + } else { + childArgs += value + parse(tail) + } + + case Nil => + } } private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { @@ -171,18 +279,30 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { outStream.println("Unknown/unsupported param " + unknownParam) } outStream.println( - """Usage: spark-submit [options] + """Usage: spark-submit [options] [app options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. | --class CLASS_NAME Name of your app's main class (required for Java apps). | --arg ARG Argument to be passed to your application's main class. This | option can be specified multiple times for multiple args. - | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). | --name NAME The name of your application (Default: 'Spark'). | --jars JARS A comma-separated list of local jars to include on the | driver classpath and that SparkContext.addJar will work | with. Doesn't work on standalone with 'cluster' deploy mode. + | --files FILES Comma separated list of files to be placed in the working dir + | of each executor. + | --properties-file FILE Path to a file from which to load extra properties. If not + | specified, this will look for conf/spark-defaults.conf. + | + | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). + | --driver-java-options Extra Java options to pass to the driver + | --driver-library-path Extra library path entries to pass to the driver + | --driver-class-path Extra class path entries to pass to the driver. Note that + | jars added with --jars are automatically included in the + | classpath. + | + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). @@ -193,14 +313,28 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | | YARN-only: | --executor-cores NUM Number of cores per executor (Default: 1). - | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | --queue QUEUE_NAME The YARN queue to submit to (Default: 'default'). | --num-executors NUM Number of executors to (Default: 2). - | --files FILES Comma separated list of files to be placed in the working dir - | of each executor. | --archives ARCHIVES Comma separated list of archives to be extracted into the | working dir of each executor.""".stripMargin ) SparkSubmit.exitFn() } } + +object SparkSubmitArguments { + /** Load properties present in the given file. */ + def getPropertiesFromFile(file: File): Seq[(String, String)] = { + require(file.exists(), s"Properties file ${file.getName} does not exist") + val inputStream = new FileInputStream(file) + val properties = new Properties() + try { + properties.load(inputStream) + } catch { + case e: IOException => + val message = s"Failed when loading Spark properties file ${file.getName}" + throw new SparkException(message, e) + } + properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 63f166d401059..888dd45e93c6a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -49,8 +49,8 @@ private[spark] object TestClient { val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription( - "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), - Some("dummy-spark-home"), "ignored") + "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), + Seq()), Some("dummy-spark-home"), "ignored") val listener = new TestListener val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf) client.start() 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 cf64700f9098c..1238bbf9da2fd 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 @@ -22,6 +22,7 @@ import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ import org.apache.spark.ui.{WebUI, SparkUI} import org.apache.spark.ui.JettyUtils._ @@ -98,6 +99,11 @@ class HistoryServer( def initialize() { attachPage(new HistoryPage(this)) attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) + } + + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + super.bind() logCheckingThread.start() } @@ -162,17 +168,21 @@ class HistoryServer( * directory. If this file exists, the associated application is regarded to be completed, in * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. */ - private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) { + private def renderSparkUI(logDir: FileStatus, elogInfo: EventLoggingInfo) { val path = logDir.getPath val appId = path.getName - val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) + val replayBus = new ReplayListenerBus(elogInfo.logPaths, fileSystem, elogInfo.compressionCodec) val appListener = new ApplicationEventListener replayBus.addListener(appListener) - val ui = new SparkUI(conf, replayBus, appId, "/history/" + appId) + val appConf = conf.clone() + val appSecManager = new SecurityManager(appConf) + val ui = new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId) // Do not call ui.bind() to avoid creating a new server for each application replayBus.replay() if (appListener.applicationStarted) { + appSecManager.setUIAcls(HISTORY_UI_ACLS_ENABLED) + appSecManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) attachSparkUI(ui) val appName = appListener.appName val sparkUser = appListener.sparkUser @@ -196,6 +206,7 @@ class HistoryServer( private def attachSparkUI(ui: SparkUI) { assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") ui.getHandlers.foreach(attachHandler) + addFilters(ui.getHandlers, conf) } /** Detach a reconstructed UI from this server. Only valid after bind(). */ @@ -249,9 +260,13 @@ object HistoryServer { // The port to which the web UI is bound val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) + // set whether to enable or disable view acls for all applications + val HISTORY_UI_ACLS_ENABLED = conf.getBoolean("spark.history.ui.acls.enable", false) + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR def main(argStrings: Array[String]) { + initSecurity() val args = new HistoryServerArguments(argStrings) val securityManager = new SecurityManager(conf) val server = new HistoryServer(args.logDir, securityManager, conf) @@ -261,6 +276,20 @@ object HistoryServer { while(true) { Thread.sleep(Int.MaxValue) } server.stop() } + + def initSecurity() { + // If we are accessing HDFS and it has security enabled (Kerberos), we have to login + // from a keytab file so that we can access HDFS beyond the kerberos ticket expiration. + // As long as it is using Hadoop rpc (hdfs://), a relogin will automatically + // occur from the keytab. + if (conf.getBoolean("spark.history.kerberos.enabled", false)) { + // if you have enabled kerberos the following 2 params must be set + val principalName = conf.get("spark.history.kerberos.principal") + val keytabFilename = conf.get("spark.history.kerberos.keytab") + SparkHadoopUtil.get.loginUserFromKeytab(principalName, keytabFilename) + } + } + } 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 81f990bfa6513..fdb633bd33608 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 @@ -237,8 +237,7 @@ private[spark] class Master( if (waitingDrivers.contains(d)) { waitingDrivers -= d self ! DriverStateChanged(driverId, DriverState.KILLED, None) - } - else { + } else { // We just notify the worker to kill the driver here. The final bookkeeping occurs // on the return path when the worker submits a state change back to the master // to notify it that the driver was successfully killed. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 0c761dfc93a1f..3e615e753b342 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -46,21 +46,27 @@ object CommandUtils extends Logging { * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val libraryOpts = getEnv("SPARK_LIBRARY_PATH", command) - .map(p => List("-Djava.library.path=" + p)) - .getOrElse(Nil) - val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")) - .map(Utils.splitCommandString).getOrElse(Nil) - val userOpts = getEnv("SPARK_JAVA_OPTS", command).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") + // Note, this will coalesce multiple options into a single command component + val extraOpts = command.extraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq()) + + val libraryOpts = + if (command.libraryPathEntries.size > 0) { + val joined = command.libraryPathEntries.mkString(File.pathSeparator) + Seq(s"-Djava.library.path=$joined") + } else { + Seq() + } // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( Seq(sparkHome + "/bin/compute-classpath" + ext), extraEnvironment=command.environment) + val userClassPath = command.classPathEntries ++ Seq(classPath) - Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts + Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ + libraryOpts ++ extraOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index b4df1a0dd4718..662d37871e7a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker import java.io._ import scala.collection.JavaConversions._ -import scala.collection.mutable.Map +import scala.collection.Map import akka.actor.ActorRef import com.google.common.base.Charsets @@ -74,22 +74,28 @@ private[spark] class DriverRunner( // Make sure user application jar is on the classpath // TODO: If we add ability to submit multiple jars they should also be added here - val env = Map(driverDesc.command.environment.toSeq: _*) - env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename" - val newCommand = Command(driverDesc.command.mainClass, - driverDesc.command.arguments.map(substituteVariables), env) + val classPath = driverDesc.command.classPathEntries ++ Seq(s"$localJarFilename") + val newCommand = Command( + driverDesc.command.mainClass, + driverDesc.command.arguments.map(substituteVariables), + driverDesc.command.environment, + classPath, + driverDesc.command.libraryPathEntries, + driverDesc.command.extraJavaOptions) val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, sparkHome.getAbsolutePath) - launchDriver(command, env, driverDir, driverDesc.supervise) + launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) } catch { case e: Exception => finalException = Some(e) } val state = - if (killed) { DriverState.KILLED } - else if (finalException.isDefined) { DriverState.ERROR } - else { + if (killed) { + DriverState.KILLED + } else if (finalException.isDefined) { + DriverState.ERROR + } else { finalExitCode match { case Some(0) => DriverState.FINISHED case _ => DriverState.FAILED 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 2edd921066876..2051403682737 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 @@ -58,30 +58,29 @@ private[spark] class ExecutorRunner( override def run() { fetchAndRunExecutor() } } workerThread.start() - // Shutdown hook that kills actors on shutdown. shutdownHook = new Thread() { override def run() { - if (process != null) { - logInfo("Shutdown hook killing child process.") - process.destroy() - process.waitFor() - } + killProcess() } } Runtime.getRuntime.addShutdownHook(shutdownHook) } + private def killProcess() { + if (process != null) { + logInfo("Killing process!") + process.destroy() + process.waitFor() + } + } + /** Stop this executor runner, including killing the process it launched */ def kill() { if (workerThread != null) { + // the workerThread will kill the child process when interrupted workerThread.interrupt() workerThread = null - if (process != null) { - logInfo("Killing process!") - process.destroy() - process.waitFor() - } state = ExecutorState.KILLED worker ! ExecutorStateChanged(appId, execId, state, None, None) Runtime.getRuntime.removeShutdownHook(shutdownHook) @@ -99,7 +98,9 @@ private[spark] class ExecutorRunner( def getCommandSeq = { val command = Command(appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment) + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment, + appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, + appDesc.command.extraJavaOptions) CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) } @@ -126,7 +127,6 @@ private[spark] class ExecutorRunner( // parent process for the executor command env.put("SPARK_LAUNCH_WITH_SCALA", "0") process = builder.start() - val header = "Spark Executor Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) @@ -146,14 +146,13 @@ private[spark] class ExecutorRunner( val message = "Command exited with code " + exitCode worker ! ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode)) } catch { - case interrupted: InterruptedException => + case interrupted: InterruptedException => { logInfo("Runner thread for executor " + fullId + " interrupted") - + killProcess() + } case e: Exception => { logError("Error running executor", e) - if (process != null) { - process.destroy() - } + killProcess() state = ExecutorState.FAILED val message = e.getClass + ": " + e.getMessage worker ! ExecutorStateChanged(appId, execId, state, Some(message), None) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index fec1207948628..8381f59672ea3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -89,8 +89,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { Previous {Utils.bytesToString(math.min(byteLength, startByte))} - } - else { + } else { @@ -104,8 +103,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { Next {Utils.bytesToString(math.min(byteLength, logLength - endByte))} - } - else { + } else { @@ -137,9 +135,13 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { val logLength = file.length() val getOffset = offset.getOrElse(logLength - defaultBytes) val startByte = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset + if (getOffset < 0) { + 0L + } else if (getOffset > logLength) { + logLength + } else { + getOffset + } val logPageLength = math.min(byteLength, maxBytes) val endByte = math.min(startByte + logPageLength, logLength) (startByte, endByte) 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 6327ac01663f6..9ac7365f47f9f 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -69,12 +69,12 @@ private[spark] class CoarseGrainedExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case KillTask(taskId, _) => + case KillTask(taskId, _, interruptThread) => if (executor == null) { logError("Received KillTask command but executor was null") System.exit(1) } else { - executor.killTask(taskId) + executor.killTask(taskId, interruptThread) } case x: DisassociatedEvent => 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 f89b2bffd1676..272bcda5f8f2f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -64,9 +64,10 @@ private[spark] class Executor( // to what Yarn on this system said was available. This will be used later when SparkEnv // created. if (java.lang.Boolean.valueOf( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) - { + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) { conf.set("spark.local.dir", getYarnLocalDirs()) + } else if (sys.env.contains("SPARK_LOCAL_DIRS")) { + conf.set("spark.local.dir", sys.env("SPARK_LOCAL_DIRS")) } if (!isLocal) { @@ -135,10 +136,10 @@ private[spark] class Executor( threadPool.execute(tr) } - def killTask(taskId: Long) { + def killTask(taskId: Long, interruptThread: Boolean) { val tr = runningTasks.get(taskId) if (tr != null) { - tr.kill() + tr.kill(interruptThread) } } @@ -160,16 +161,14 @@ private[spark] class Executor( class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) extends Runnable { - object TaskKilledException extends Exception - @volatile private var killed = false @volatile private var task: Task[Any] = _ - def kill() { + def kill(interruptThread: Boolean) { logInfo("Executor is trying to kill task " + taskId) killed = true if (task != null) { - task.kill() + task.kill(interruptThread) } } @@ -199,7 +198,7 @@ private[spark] class Executor( // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl // exception will be caught by the catch block, leading to an incorrect ExceptionFailure // for the task. - throw TaskKilledException + throw new TaskKilledException } attemptedTask = Some(task) @@ -213,7 +212,7 @@ private[spark] class Executor( // If the task has been killed, let's fail it. if (task.killed) { - throw TaskKilledException + throw new TaskKilledException } val resultSer = SparkEnv.get.serializer.newInstance() @@ -256,7 +255,7 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) } - case TaskKilledException => { + case _: TaskKilledException | _: InterruptedException if task.killed => { logInfo("Executor killed task " + taskId) execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) } 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 6fc702fdb1512..64e24506e8038 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -76,7 +76,8 @@ private[spark] class MesosExecutorBackend if (executor == null) { logError("Received KillTask but executor was null") } else { - executor.killTask(t.getValue.toLong) + // TODO: Determine the 'interruptOnCancel' property set for the given job. + executor.killTask(t.getValue.toLong, interruptThread = false) } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index e4f02a4be0b97..350fd74173f65 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -84,7 +84,7 @@ class TaskMetrics extends Serializable { } private[spark] object TaskMetrics { - def empty(): TaskMetrics = new TaskMetrics + def empty: TaskMetrics = new TaskMetrics } 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 c5bda2078fc14..651511da1b7fe 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -129,17 +129,19 @@ private[spark] class MetricsSystem private (val instance: String, sinkConfigs.foreach { kv => val classPath = kv._2.getProperty("class") - try { - val sink = Class.forName(classPath) - .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) - .newInstance(kv._2, registry, securityMgr) - if (kv._1 == "servlet") { - metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) - } else { - sinks += sink.asInstanceOf[Sink] + if (null != classPath) { + try { + val sink = Class.forName(classPath) + .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) + .newInstance(kv._2, registry, securityMgr) + if (kv._1 == "servlet") { + metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) + } else { + sinks += sink.asInstanceOf[Sink] + } + } catch { + case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } - } catch { - case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } } } diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 9dc51e0d401f8..53a6038a9b59e 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -28,7 +28,7 @@ private[spark] object ReceiverTest { manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */ - val buffer = ByteBuffer.wrap("response".getBytes) + val buffer = ByteBuffer.wrap("response".getBytes("utf-8")) Some(Message.createBufferMessage(buffer, msg.id)) }) Thread.currentThread.join() diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 14c094c6177d5..b8ea7c2cff9a2 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -54,7 +54,7 @@ private[spark] object SenderTest { val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) - new String(buffer.array) + new String(buffer.array, "utf-8") }.getOrElse("none") val finishTime = System.currentTimeMillis diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 59bbb1171f239..5cdbc306e56a0 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -30,7 +30,7 @@ package org.apache * type (e.g. RDD[(Int, Int)] through implicit conversions when you * `import org.apache.spark.SparkContext._`. * - * Java programmers should reference the [[spark.api.java]] package + * Java programmers should reference the [[org.apache.spark.api.java]] package * for Spark programming APIs in Java. * * Classes and methods marked with diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index e6c4a6d3794a0..c64da8804d166 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -19,24 +19,30 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} +import org.apache.spark._ import org.apache.spark.storage.{BlockId, BlockManager} +import scala.Some private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition { val index = idx } private[spark] -class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId]) +class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds: Array[BlockId]) extends RDD[T](sc, Nil) { @transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get) + @volatile private var _isValid = true - override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => { - new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] - }).toArray + override def getPartitions: Array[Partition] = { + assertValid() + (0 until blockIds.size).map(i => { + new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] + }).toArray + } override def compute(split: Partition, context: TaskContext): Iterator[T] = { + assertValid() val blockManager = SparkEnv.get.blockManager val blockId = split.asInstanceOf[BlockRDDPartition].blockId blockManager.get(blockId) match { @@ -47,7 +53,36 @@ class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId } override def getPreferredLocations(split: Partition): Seq[String] = { + assertValid() locations_(split.asInstanceOf[BlockRDDPartition].blockId) } + + /** + * Remove the data blocks that this BlockRDD is made from. NOTE: This is an + * irreversible operation, as the data in the blocks cannot be recovered back + * once removed. Use it with caution. + */ + private[spark] def removeBlocks() { + blockIds.foreach { blockId => + sc.env.blockManager.master.removeBlock(blockId) + } + _isValid = false + } + + /** + * Whether this BlockRDD is actually usable. This will be false if the data blocks have been + * removed using `this.removeBlocks`. + */ + private[spark] def isValid: Boolean = { + _isValid + } + + /** Check if this BlockRDD is valid. If not valid, exception is thrown. */ + private[spark] def assertValid() { + if (!_isValid) { + throw new SparkException( + "Attempted to use %s after its blocks have been removed!".format(toString)) + } + } } 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 d250bef6aad0f..5efb4388f6c71 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -52,11 +52,12 @@ import org.apache.spark.util.SerializableHyperLogLog * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ -class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) +class PairRDDFunctions[K, V](self: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) extends Logging with SparkHadoopMapReduceUtil - with Serializable { - + with Serializable +{ /** * Generic function to combine the elements for each key using a custom set of aggregation * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C @@ -77,7 +78,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) mapSideCombine: Boolean = true, serializer: Serializer = null): RDD[(K, C)] = { require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0 - if (getKeyClass().isArray) { + if (keyClass.isArray) { if (mapSideCombine) { throw new SparkException("Cannot use map-side combining with array keys.") } @@ -170,7 +171,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = { - if (getKeyClass().isArray) { + if (keyClass.isArray) { throw new SparkException("reduceByKeyLocally() does not support array keys") } @@ -288,10 +289,14 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Return a copy of the RDD partitioned using the specified partitioner. */ def partitionBy(partitioner: Partitioner): RDD[(K, V)] = { - if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { + if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } - if (self.partitioner == partitioner) self else new ShuffledRDD[K, V, (K, V)](self, partitioner) + if (self.partitioner == Some(partitioner)) { + self + } else { + new ShuffledRDD[K, V, (K, V)](self, partitioner) + } } /** @@ -458,7 +463,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) @@ -473,7 +478,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) @@ -573,7 +578,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * supporting the key and value types K and V in this RDD. */ def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) + saveAsHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -584,7 +589,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def saveAsHadoopFile[F <: OutputFormat[K, V]]( path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { val runtimeClass = fm.runtimeClass - saveAsHadoopFile(path, getKeyClass, getValueClass, runtimeClass.asInstanceOf[Class[F]], codec) + saveAsHadoopFile(path, keyClass, valueClass, runtimeClass.asInstanceOf[Class[F]], codec) } /** @@ -592,7 +597,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. */ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) + saveAsNewAPIHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -782,7 +787,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def values: RDD[V] = self.map(_._2) - private[spark] def getKeyClass() = implicitly[ClassTag[K]].runtimeClass + private[spark] def keyClass: Class[_] = kt.runtimeClass + + private[spark] def valueClass: Class[_] = vt.runtimeClass - private[spark] def getValueClass() = implicitly[ClassTag[V]].runtimeClass + private[spark] def keyOrdering: Option[Ordering[K]] = Option(ord) } diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index b4e3bb5d75e17..b5b8a5706deb3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, TaskContext} import org.apache.spark.util.random.RandomSampler +import org.apache.spark.util.Utils private[spark] class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) @@ -38,14 +39,14 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * * @param prev RDD to be sampled * @param sampler a random sampler - * @param seed random seed, default to System.nanoTime + * @param seed random seed * @tparam T input RDD item type * @tparam U sampled RDD item type */ private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], - @transient seed: Long = System.nanoTime) + @transient seed: Long = Utils.random.nextLong) extends RDD[U](prev) { override def getPartitions: Array[Partition] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index e441d4a40ccd2..5d77d37378458 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -87,10 +87,10 @@ private[spark] class PipedRDD[T: ClassTag]( // When spark.worker.separated.working.directory option is turned on, each // task will be run in separate directory. This should be resolve file // access conflict issue - val taskDirectory = "./tasks/" + java.util.UUID.randomUUID.toString + val taskDirectory = "tasks" + File.separator + java.util.UUID.randomUUID.toString var workInTaskDirectory = false logDebug("taskDirectory = " + taskDirectory) - if (separateWorkingDir == true) { + if (separateWorkingDir) { val currentDir = new File(".") logDebug("currentDir = " + currentDir.getAbsolutePath()) val taskDirFile = new File(taskDirectory) @@ -106,13 +106,13 @@ private[spark] class PipedRDD[T: ClassTag]( for (file <- currentDir.list(tasksDirFilter)) { val fileWithDir = new File(currentDir, file) Utils.symlink(new File(fileWithDir.getAbsolutePath()), - new File(taskDirectory + "/" + fileWithDir.getName())) + new File(taskDirectory + File.separator + fileWithDir.getName())) } pb.directory(taskDirFile) workInTaskDirectory = true } catch { case e: Exception => logError("Unable to setup task working directory: " + e.getMessage + - " (" + taskDirectory + ")") + " (" + taskDirectory + ")", e) } } 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 891efccf23b6a..3b3524f33e811 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.util.Random import scala.collection.Map +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} @@ -94,26 +95,20 @@ abstract class RDD[T: ClassTag]( def compute(split: Partition, context: TaskContext): Iterator[T] /** - * :: DeveloperApi :: * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ - @DeveloperApi protected def getPartitions: Array[Partition] /** - * :: DeveloperApi :: * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ - @DeveloperApi protected def getDependencies: Seq[Dependency[_]] = deps /** - * :: DeveloperApi :: * Optionally overridden by subclasses to specify placement preferences. */ - @DeveloperApi protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -235,6 +230,30 @@ abstract class RDD[T: ClassTag]( } } + /** + * Return the ancestors of the given RDD that are related to it only through a sequence of + * narrow dependencies. This traverses the given RDD's dependency tree using DFS, but maintains + * no ordering on the RDDs returned. + */ + private[spark] def getNarrowAncestors: Seq[RDD[_]] = { + val ancestors = new mutable.HashSet[RDD[_]] + + def visit(rdd: RDD[_]) { + val narrowDependencies = rdd.dependencies.filter(_.isInstanceOf[NarrowDependency[_]]) + val narrowParents = narrowDependencies.map(_.rdd) + val narrowParentsNotVisited = narrowParents.filterNot(ancestors.contains) + narrowParentsNotVisited.foreach { parent => + ancestors.add(parent) + visit(parent) + } + } + + visit(this) + + // In case there is a cycle, do not include the root itself + ancestors.filterNot(_ == this).toSeq + } + /** * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing. */ @@ -265,7 +284,7 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numPartitions: Int): RDD[T] = + def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1) /** @@ -282,7 +301,7 @@ abstract class RDD[T: ClassTag]( * If you are decreasing the number of partitions in this RDD, consider using `coalesce`, * which can avoid performing a shuffle. */ - def repartition(numPartitions: Int): RDD[T] = { + def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = { coalesce(numPartitions, shuffle = true) } @@ -306,7 +325,8 @@ abstract class RDD[T: ClassTag]( * coalesce(1000, shuffle = true) will result in 1000 partitions with the * data distributed using a hash partitioner. */ - def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = { + def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) + : RDD[T] = { if (shuffle) { // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( @@ -321,7 +341,9 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] = { + def sample(withReplacement: Boolean, + fraction: Double, + seed: Long = Utils.random.nextLong): RDD[T] = { require(fraction >= 0.0, "Invalid fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) @@ -334,11 +356,11 @@ abstract class RDD[T: ClassTag]( * Randomly splits this RDD with the provided weights. * * @param weights weights for splits, will be normalized if they don't sum to 1 - * @param seed random seed, default to System.nanoTime + * @param seed random seed * * @return split RDDs in an array */ - def randomSplit(weights: Array[Double], seed: Long = System.nanoTime): Array[RDD[T]] = { + def randomSplit(weights: Array[Double], seed: Long = Utils.random.nextLong): Array[RDD[T]] = { val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => @@ -346,7 +368,8 @@ abstract class RDD[T: ClassTag]( }.toArray } - def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { + def takeSample(withReplacement: Boolean, num: Int, seed: Long = Utils.random.nextLong): Array[T] = + { var fraction = 0.0 var total = 0 val multiplier = 3.0 @@ -405,10 +428,11 @@ abstract class RDD[T: ClassTag]( * * Note that this method performs a shuffle internally. */ - def intersection(other: RDD[T]): RDD[T] = + def intersection(other: RDD[T]): RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null))) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return the intersection of this RDD and another one. The output will not contain any duplicate @@ -418,10 +442,12 @@ abstract class RDD[T: ClassTag]( * * @param partitioner Partitioner to use for the resulting RDD */ - def intersection(other: RDD[T], partitioner: Partitioner): RDD[T] = + def intersection(other: RDD[T], partitioner: Partitioner)(implicit ord: Ordering[T] = null) + : RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null)), partitioner) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return the intersection of this RDD and another one. The output will not contain any duplicate @@ -431,10 +457,11 @@ abstract class RDD[T: ClassTag]( * * @param numPartitions How many partitions to use in the resulting RDD */ - def intersection(other: RDD[T], numPartitions: Int): RDD[T] = + def intersection(other: RDD[T], numPartitions: Int): RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null)), new HashPartitioner(numPartitions)) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return an RDD created by coalescing all elements within each partition into an array. @@ -448,22 +475,25 @@ abstract class RDD[T: ClassTag]( def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) /** - * Return an RDD of grouped items. + * Return an RDD of grouped items. Each group consists of a key and a sequence of elements + * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K): RDD[(K, Iterable[T])] = + def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Iterable[T])] = + def groupBy[K](f: T => K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** - * Return an RDD of grouped items. + * Return an RDD of grouped items. Each group consists of a key and a sequence of elements + * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Iterable[T])] = { + def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null) + : RDD[(K, Iterable[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } @@ -720,7 +750,7 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD with the elements from `this` that are not in `other`. */ - def subtract(other: RDD[T], p: Partitioner): RDD[T] = { + def subtract(other: RDD[T], p: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = { if (partitioner == Some(p)) { // Our partitioner knows how to handle T (which, since we have a partitioner, is // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples @@ -828,7 +858,7 @@ abstract class RDD[T: ClassTag]( * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final * combine step happens locally on the master, equivalent to running a single reduce task. */ - def countByValue(): Map[T, Long] = { + def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = { if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValue() does not support arrays") } @@ -858,10 +888,10 @@ abstract class RDD[T: ClassTag]( * Approximate version of countByValue(). */ @Experimental - def countByValueApprox( - timeout: Long, - confidence: Double = 0.95 - ): PartialResult[Map[T, BoundedDouble]] = { + def countByValueApprox(timeout: Long, confidence: Double = 0.95) + (implicit ord: Ordering[T] = null) + : PartialResult[Map[T, BoundedDouble]] = + { if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } @@ -1011,13 +1041,13 @@ abstract class RDD[T: ClassTag]( * Returns the max of this RDD as defined by the implicit Ordering[T]. * @return the maximum element of the RDD * */ - def max()(implicit ord: Ordering[T]):T = this.reduce(ord.max) + def max()(implicit ord: Ordering[T]): T = this.reduce(ord.max) /** * Returns the min of this RDD as defined by the implicit Ordering[T]. * @return the minimum element of the RDD * */ - def min()(implicit ord: Ordering[T]):T = this.reduce(ord.min) + def min()(implicit ord: Ordering[T]): T = this.reduce(ord.min) /** * Save this RDD as a text file, using string representations of elements. @@ -1112,9 +1142,9 @@ abstract class RDD[T: ClassTag]( @transient private var doCheckpointCalled = false /** - * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler - * after a job using this RDD has completed (therefore the RDD has been materialized and - * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs. + * Performs the checkpointing of this RDD by saving this. It is called after a job using this RDD + * has completed (therefore the RDD has been materialized and potentially stored in memory). + * doCheckpoint() is called recursively on the parent RDDs. */ private[spark] def doCheckpoint() { if (!doCheckpointCalled) { diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 7df9a2960d8a5..9a1efc83cbe6a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -68,8 +68,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag val keyClass = getWritableClass[K] val valueClass = getWritableClass[V] - val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass) - val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass) + val convertKey = !classOf[Writable].isAssignableFrom(self.keyClass) + val convertValue = !classOf[Writable].isAssignableFrom(self.valueClass) logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index c1001227151a5..cd5d44ad4a7e6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -28,6 +28,8 @@ private[spark] class ApplicationEventListener extends SparkListener { var sparkUser = "" var startTime = -1L var endTime = -1L + var viewAcls = "" + var enableViewAcls = false def applicationStarted = startTime != -1 @@ -47,4 +49,13 @@ private[spark] class ApplicationEventListener extends SparkListener { override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { endTime = applicationEnd.time } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + val allProperties = environmentDetails("Spark Properties").toMap + viewAcls = allProperties.getOrElse("spark.ui.view.acls", "") + enableViewAcls = allProperties.getOrElse("spark.ui.acls.enable", "false").toBoolean + } + } } 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 c6cbf14e20069..ff411e24a3d85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -22,10 +22,16 @@ import java.util.Properties import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.postfixOps import scala.reflect.ClassTag import akka.actor._ +import akka.actor.OneForOneStrategy +import akka.actor.SupervisorStrategy.Stop +import akka.pattern.ask +import akka.util.Timeout import org.apache.spark._ import org.apache.spark.executor.TaskMetrics @@ -47,14 +53,11 @@ import org.apache.spark.util.Utils * not caused by shuffle file loss are handled by the TaskScheduler, which will retry each task * a small number of times before cancelling the whole stage. * - * THREADING: This class runs all its logic in a single thread executing the run() method, to which - * events are submitted using a synchronized queue (eventQueue). The public API methods, such as - * runJob, taskEnded and executorLost, post events asynchronously to this queue. All other methods - * should be private. */ private[spark] class DAGScheduler( - taskScheduler: TaskScheduler, + private[scheduler] val sc: SparkContext, + private[scheduler] val taskScheduler: TaskScheduler, listenerBus: LiveListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, @@ -65,6 +68,7 @@ class DAGScheduler( def this(sc: SparkContext, taskScheduler: TaskScheduler) = { this( + sc, taskScheduler, sc.listenerBus, sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], @@ -74,8 +78,6 @@ class DAGScheduler( def this(sc: SparkContext) = this(sc, sc.taskScheduler) - private var eventProcessActor: ActorRef = _ - private[scheduler] val nextJobId = new AtomicInteger(0) private[scheduler] def numTotalJobs: Int = nextJobId.get() private val nextStageId = new AtomicInteger(0) @@ -113,50 +115,31 @@ class DAGScheduler( // stray messages to detect. private val failedEpoch = new HashMap[String, Long] - taskScheduler.setDAGScheduler(this) + private val dagSchedulerActorSupervisor = + env.actorSystem.actorOf(Props(new DAGSchedulerActorSupervisor(this))) - /** - * Starts the event processing actor. The actor has two responsibilities: - * - * 1. Waits for events like job submission, task finished, task failure etc., and calls - * [[org.apache.spark.scheduler.DAGScheduler.processEvent()]] to process them. - * 2. Schedules a periodical task to resubmit failed stages. - * - * NOTE: the actor cannot be started in the constructor, because the periodical task references - * some internal states of the enclosing [[org.apache.spark.scheduler.DAGScheduler]] object, thus - * cannot be scheduled until the [[org.apache.spark.scheduler.DAGScheduler]] is fully constructed. - */ - def start() { - eventProcessActor = env.actorSystem.actorOf(Props(new Actor { - /** - * The main event loop of the DAG scheduler. - */ - def receive = { - case event: DAGSchedulerEvent => - logTrace("Got event of type " + event.getClass.getName) - - /** - * All events are forwarded to `processEvent()`, so that the event processing logic can - * easily tested without starting a dedicated actor. Please refer to `DAGSchedulerSuite` - * for details. - */ - if (!processEvent(event)) { - submitWaitingStages() - } else { - context.stop(self) - } - } - })) + private[scheduler] var eventProcessActor: ActorRef = _ + + private def initializeEventProcessActor() { + // blocking the thread until supervisor is started, which ensures eventProcessActor is + // not null before any job is submitted + implicit val timeout = Timeout(30 seconds) + val initEventActorReply = + dagSchedulerActorSupervisor ? Props(new DAGSchedulerEventProcessActor(this)) + eventProcessActor = Await.result(initEventActorReply, timeout.duration). + asInstanceOf[ActorRef] } + initializeEventProcessActor() + // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventProcessActor ! BeginEvent(task, taskInfo) } // Called to report that a task has completed and results are being fetched remotely. - def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! GettingResultEvent(task, taskInfo) + def taskGettingResult(taskInfo: TaskInfo) { + eventProcessActor ! GettingResultEvent(taskInfo) } // Called by TaskScheduler to report task completions or failures. @@ -436,7 +419,7 @@ class DAGScheduler( { // Check to make sure we are not launching a task on a partition that does not exist. val maxPartitions = rdd.partitions.length - partitions.find(p => p >= maxPartitions).foreach { p => + partitions.find(p => p >= maxPartitions || p < 0).foreach { p => throw new IllegalArgumentException( "Attempting to access a non-existent partition: " + p + ". " + "Total number of partitions: " + maxPartitions) @@ -511,6 +494,15 @@ class DAGScheduler( eventProcessActor ! AllJobsCancelled } + private[scheduler] def doCancelAllJobs() { + // Cancel all running jobs. + runningStages.map(_.jobId).foreach(handleJobCancellation(_, + reason = "as part of cancellation of all jobs")) + activeJobs.clear() // These should already be empty by this point, + jobIdToActiveJob.clear() // but just in case we lost track of some jobs... + submitWaitingStages() + } + /** * Cancel all jobs associated with a running or scheduled stage. */ @@ -518,148 +510,30 @@ class DAGScheduler( eventProcessActor ! StageCancelled(stageId) } - /** - * Process one event retrieved from the event processing actor. - * - * @param event The event to be processed. - * @return `true` if we should stop the event loop. - */ - private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { - event match { - case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => - var finalStage: Stage = null - try { - // New stage creation may throw an exception if, for example, jobs are run on a HadoopRDD - // whose underlying HDFS files have been deleted. - finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) - } catch { - case e: Exception => - logWarning("Creating new stage failed due to exception - job: " + jobId, e) - listener.jobFailed(e) - return false - } - val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) - clearCacheLocs() - logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length + - " output partitions (allowLocal=" + allowLocal + ")") - logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")") - logInfo("Parents of final stage: " + finalStage.parents) - logInfo("Missing parents: " + getMissingParentStages(finalStage)) - if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { - // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) - runLocally(job) - } else { - jobIdToActiveJob(jobId) = job - activeJobs += job - resultStageToJob(finalStage) = job - listenerBus.post( - SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) - submitStage(finalStage) - } - - case StageCancelled(stageId) => - handleStageCancellation(stageId) - - case JobCancelled(jobId) => - handleJobCancellation(jobId) - - case JobGroupCancelled(groupId) => - // Cancel all jobs belonging to this job group. - // First finds all active jobs with this group id, and then kill stages for them. - val activeInGroup = activeJobs.filter(activeJob => - groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) - val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach(jobId => handleJobCancellation(jobId, - "as part of cancelled job group %s".format(groupId))) - - case AllJobsCancelled => - // Cancel all running jobs. - runningStages.map(_.jobId).foreach(jobId => handleJobCancellation(jobId, - "as part of cancellation of all jobs")) - activeJobs.clear() // These should already be empty by this point, - jobIdToActiveJob.clear() // but just in case we lost track of some jobs... - - case ExecutorAdded(execId, host) => - handleExecutorAdded(execId, host) - - case ExecutorLost(execId) => - handleExecutorLost(execId) - - case BeginEvent(task, taskInfo) => - for ( - stage <- stageIdToStage.get(task.stageId); - stageInfo <- stageToInfos.get(stage) - ) { - if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 && - !stageInfo.emittedTaskSizeWarning) { - stageInfo.emittedTaskSizeWarning = true - logWarning(("Stage %d (%s) contains a task of very large " + - "size (%d KB). The maximum recommended task size is %d KB.").format( - task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) - } - } - listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) - - case GettingResultEvent(task, taskInfo) => - listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) - - case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => - val stageId = task.stageId - val taskType = Utils.getFormattedClassName(task) - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) - handleTaskCompletion(completion) - - case TaskSetFailed(taskSet, reason) => - stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } - - case ResubmitFailedStages => - if (failedStages.size > 0) { - // Failed stages may be removed by job cancellation, so failed might be empty even if - // the ResubmitFailedStages event has been scheduled. - resubmitFailedStages() - } - - case StopDAGScheduler => - // Cancel any active jobs - for (job <- activeJobs) { - val error = new SparkException("Job cancelled because SparkContext was shut down") - job.listener.jobFailed(error) - // Tell the listeners that all of the running stages have ended. Don't bother - // cancelling the stages because if the DAG scheduler is stopped, the entire application - // is in the process of getting stopped. - val stageFailedMessage = "Stage cancelled because SparkContext was shut down" - runningStages.foreach { stage => - val info = stageToInfos(stage) - info.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(info)) - } - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) - } - return true - } - false - } - /** * Resubmit any failed stages. Ordinarily called after a small amount of time has passed since * the last fetch failure. */ private[scheduler] def resubmitFailedStages() { - logInfo("Resubmitting failed stages") - clearCacheLocs() - val failedStagesCopy = failedStages.toArray - failedStages.clear() - for (stage <- failedStagesCopy.sortBy(_.jobId)) { - submitStage(stage) + if (failedStages.size > 0) { + // Failed stages may be removed by job cancellation, so failed might be empty even if + // the ResubmitFailedStages event has been scheduled. + logInfo("Resubmitting failed stages") + clearCacheLocs() + val failedStagesCopy = failedStages.toArray + failedStages.clear() + for (stage <- failedStagesCopy.sortBy(_.jobId)) { + submitStage(stage) + } } + submitWaitingStages() } /** * Check for waiting or failed stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. */ - private[scheduler] def submitWaitingStages() { + private def submitWaitingStages() { // TODO: We might want to run this less often, when we are sure that something has become // runnable that wasn't before. logTrace("Checking for newly runnable parent stages") @@ -730,6 +604,102 @@ class DAGScheduler( } } + private[scheduler] def handleJobGroupCancelled(groupId: String) { + // Cancel all jobs belonging to this job group. + // First finds all active jobs with this group id, and then kill stages for them. + val activeInGroup = activeJobs.filter(activeJob => + groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) + val jobIds = activeInGroup.map(_.jobId) + jobIds.foreach(handleJobCancellation(_, "part of cancelled job group %s".format(groupId))) + submitWaitingStages() + } + + private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) { + for (stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage)) { + if (taskInfo.serializedSize > DAGScheduler.TASK_SIZE_TO_WARN * 1024 && + !stageInfo.emittedTaskSizeWarning) { + stageInfo.emittedTaskSizeWarning = true + logWarning(("Stage %d (%s) contains a task of very large " + + "size (%d KB). The maximum recommended task size is %d KB.").format( + task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, + DAGScheduler.TASK_SIZE_TO_WARN)) + } + } + listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) + submitWaitingStages() + } + + private[scheduler] def handleTaskSetFailed(taskSet: TaskSet, reason: String) { + stageIdToStage.get(taskSet.stageId).foreach {abortStage(_, reason) } + submitWaitingStages() + } + + private[scheduler] def cleanUpAfterSchedulerStop() { + for (job <- activeJobs) { + val error = new SparkException("Job cancelled because SparkContext was shut down") + job.listener.jobFailed(error) + // Tell the listeners that all of the running stages have ended. Don't bother + // cancelling the stages because if the DAG scheduler is stopped, the entire application + // is in the process of getting stopped. + val stageFailedMessage = "Stage cancelled because SparkContext was shut down" + runningStages.foreach { stage => + val info = stageToInfos(stage) + info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(info)) + } + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + } + } + + private[scheduler] def handleGetTaskResult(taskInfo: TaskInfo) { + listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) + submitWaitingStages() + } + + private[scheduler] def handleJobSubmitted(jobId: Int, + finalRDD: RDD[_], + func: (TaskContext, Iterator[_]) => _, + partitions: Array[Int], + allowLocal: Boolean, + callSite: String, + listener: JobListener, + properties: Properties = null) + { + var finalStage: Stage = null + try { + // New stage creation may throw an exception if, for example, jobs are run on a + // HadoopRDD whose underlying HDFS files have been deleted. + finalStage = newStage(finalRDD, partitions.size, None, jobId, Some(callSite)) + } catch { + case e: Exception => + logWarning("Creating new stage failed due to exception - job: " + jobId, e) + listener.jobFailed(e) + return + } + if (finalStage != null) { + val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) + clearCacheLocs() + logInfo("Got job %s (%s) with %d output partitions (allowLocal=%s)".format( + job.jobId, callSite, partitions.length, allowLocal)) + logInfo("Final stage: " + finalStage + "(" + finalStage.name + ")") + logInfo("Parents of final stage: " + finalStage.parents) + logInfo("Missing parents: " + getMissingParentStages(finalStage)) + if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { + // Compute very short actions like first() or take() with no parent stages locally. + listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + runLocally(job) + } else { + jobIdToActiveJob(jobId) = job + activeJobs += job + resultStageToJob(finalStage) = job + listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, + properties)) + submitStage(finalStage) + } + } + submitWaitingStages() + } + /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { val jobId = activeJobForStage(stage) @@ -819,9 +789,12 @@ class DAGScheduler( * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. */ - private def handleTaskCompletion(event: CompletionEvent) { + private[scheduler] def handleTaskCompletion(event: CompletionEvent) { val task = event.task - + val stageId = task.stageId + val taskType = Utils.getFormattedClassName(task) + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, + event.taskMetrics)) if (!stageIdToStage.contains(task.stageId)) { // Skip all the actions if the stage has been cancelled. return @@ -964,6 +937,7 @@ class DAGScheduler( // Unrecognized failure - also do nothing. If the task fails repeatedly, the TaskScheduler // will abort the job. } + submitWaitingStages() } /** @@ -973,7 +947,7 @@ class DAGScheduler( * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ - private def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { + private[scheduler] def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { failedEpoch(execId) = currentEpoch @@ -993,17 +967,19 @@ class DAGScheduler( logDebug("Additional executor lost message for " + execId + "(epoch " + currentEpoch + ")") } + submitWaitingStages() } - private def handleExecutorAdded(execId: String, host: String) { + private[scheduler] def handleExecutorAdded(execId: String, host: String) { // remove from failedEpoch(execId) ? if (failedEpoch.contains(execId)) { logInfo("Host added was in lost list earlier: " + host) failedEpoch -= execId } + submitWaitingStages() } - private def handleStageCancellation(stageId: Int) { + private[scheduler] def handleStageCancellation(stageId: Int) { if (stageIdToJobIds.contains(stageId)) { val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray jobsThatUseStage.foreach(jobId => { @@ -1012,22 +988,24 @@ class DAGScheduler( } else { logInfo("No active jobs to kill for Stage " + stageId) } + submitWaitingStages() } - private def handleJobCancellation(jobId: Int, reason: String = "") { + private[scheduler] def handleJobCancellation(jobId: Int, reason: String = "") { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { failJobAndIndependentStages(jobIdToActiveJob(jobId), "Job %d cancelled %s".format(jobId, reason), None) } + submitWaitingStages() } /** * Aborts all jobs depending on a particular Stage. This is called in response to a task set * being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. */ - private def abortStage(failedStage: Stage, reason: String) { + private[scheduler] def abortStage(failedStage: Stage, reason: String) { if (!stageIdToStage.contains(failedStage.id)) { // Skip all the actions if the stage has been removed. return @@ -1055,6 +1033,10 @@ class DAGScheduler( val error = new SparkException(failureReason) job.listener.jobFailed(error) + val shouldInterruptThread = + if (job.properties == null) false + else job.properties.getProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false").toBoolean + // Cancel all independent, running stages. val stages = jobIdToStageIds(job.jobId) if (stages.isEmpty) { @@ -1073,7 +1055,7 @@ class DAGScheduler( // This is the only job that uses this stage, so fail the stage if it is running. val stage = stageIdToStage(stageId) if (runningStages.contains(stage)) { - taskScheduler.cancelTasks(stageId) + taskScheduler.cancelTasks(stageId, shouldInterruptThread) val stageInfo = stageToInfos(stage) stageInfo.stageFailed(failureReason) listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) @@ -1152,13 +1134,88 @@ class DAGScheduler( } def stop() { - if (eventProcessActor != null) { - eventProcessActor ! StopDAGScheduler - } + logInfo("Stopping DAGScheduler") + dagSchedulerActorSupervisor ! PoisonPill taskScheduler.stop() } } +private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) + extends Actor with Logging { + + override val supervisorStrategy = + OneForOneStrategy() { + case x: Exception => + logError("eventProcesserActor failed due to the error %s; shutting down SparkContext" + .format(x.getMessage)) + dagScheduler.doCancelAllJobs() + dagScheduler.sc.stop() + Stop + } + + def receive = { + case p: Props => sender ! context.actorOf(p) + case _ => logWarning("received unknown message in DAGSchedulerActorSupervisor") + } +} + +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. + */ + def receive = { + case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => + dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, + listener, properties) + + case StageCancelled(stageId) => + dagScheduler.handleStageCancellation(stageId) + + case JobCancelled(jobId) => + dagScheduler.handleJobCancellation(jobId) + + case JobGroupCancelled(groupId) => + dagScheduler.handleJobGroupCancelled(groupId) + + case AllJobsCancelled => + dagScheduler.doCancelAllJobs() + + case ExecutorAdded(execId, host) => + dagScheduler.handleExecutorAdded(execId, host) + + case ExecutorLost(execId) => + dagScheduler.handleExecutorLost(execId) + + case BeginEvent(task, taskInfo) => + dagScheduler.handleBeginEvent(task, taskInfo) + + case GettingResultEvent(taskInfo) => + dagScheduler.handleGetTaskResult(taskInfo) + + case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => + dagScheduler.handleTaskCompletion(completion) + + case TaskSetFailed(taskSet, reason) => + dagScheduler.handleTaskSetFailed(taskSet, reason) + + case ResubmitFailedStages => + dagScheduler.resubmitFailedStages() + } + + override def postStop() { + // Cancel any active jobs in postStop hook + dagScheduler.cleanUpAfterSchedulerStop() + } +} + private[spark] object DAGScheduler { // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 0800c5684c60f..23f57441b4b11 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -57,7 +57,7 @@ private[scheduler] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] -case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent +case class GettingResultEvent(taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] case class CompletionEvent( task: Task[_], @@ -76,5 +76,3 @@ private[scheduler] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent - -private[scheduler] case object StopDAGScheduler extends DAGSchedulerEvent 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 b983c16af14f4..d822a8e55111a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -19,7 +19,9 @@ package org.apache.spark.scheduler import scala.collection.mutable +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.permission.FsPermission import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} @@ -36,7 +38,10 @@ import org.apache.spark.util.{FileLogger, JsonProtocol} * spark.eventLog.dir - Path to the directory in which events are logged. * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ -private[spark] class EventLoggingListener(appName: String, conf: SparkConf) +private[spark] class EventLoggingListener( + appName: String, + conf: SparkConf, + hadoopConfiguration: Configuration) extends SparkListener with Logging { import EventLoggingListener._ @@ -49,7 +54,8 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) val logDir = logBaseDir + "/" + name private val logger = - new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) + new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress, + shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) /** * Begin logging events. @@ -119,6 +125,9 @@ private[spark] object EventLoggingListener extends Logging { val SPARK_VERSION_PREFIX = "SPARK_VERSION_" val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + val LOG_FILE_PERMISSIONS: FsPermission = + FsPermission.createImmutable(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] diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 713aebfa3ce00..a1e21cad48b9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -207,7 +207,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val taskInfo = taskEnd.taskInfo var taskStatus = "TASK_TYPE=%s".format(taskEnd.taskType) - val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty() + val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskMetrics) diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index cbac4c13ca6fe..dec3316bf7745 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.util.concurrent.LinkedBlockingQueue +import java.util.concurrent.{LinkedBlockingQueue, Semaphore} import org.apache.spark.Logging @@ -36,16 +36,24 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false private var started = false + + // A counter that represents the number of events produced and consumed in the queue + private val eventLock = new Semaphore(0) + private val listenerThread = new Thread("SparkListenerBus") { setDaemon(true) override def run() { while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return + eventLock.acquire() + // Atomically remove and process this event + LiveListenerBus.this.synchronized { + val event = eventQueue.poll + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + Option(event).foreach(postToAll) } - postToAll(event) } } } @@ -67,7 +75,9 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { def post(event: SparkListenerEvent) { val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { + if (eventAdded) { + eventLock.release() + } else if (!queueFullErrorMessageLogged) { logError("Dropping SparkListenerEvent because no remaining room in event queue. " + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + "rate at which tasks are being started by the scheduler.") @@ -76,13 +86,13 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { } /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time + * For testing only. Wait until there are no more events in the queue, or until the specified + * time has elapsed. Return true if the queue has emptied and false is the specified time * elapsed before the queue emptied. */ def waitUntilEmpty(timeoutMillis: Int): Boolean = { val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { + while (!queueIsEmpty) { if (System.currentTimeMillis > finishTime) { return false } @@ -93,6 +103,14 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { true } + /** + * Return whether the event queue is empty. + * + * The use of synchronized here guarantees that all events that once belonged to this queue + * have already been processed by all attached listeners, if this returns true. + */ + def queueIsEmpty: Boolean = synchronized { eventQueue.isEmpty } + def stop() { if (!started) { throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index f1924a4573b21..6a6d8e609bc39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -28,5 +28,6 @@ private[spark] trait SchedulerBackend { def reviveOffers(): Unit def defaultParallelism(): Int - def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException + def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = + throw new UnsupportedOperationException } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index b85eabd6bbdbc..1ce83485f024b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -24,8 +24,12 @@ import org.apache.spark.annotation.DeveloperApi // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. @DeveloperApi -class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, - val length: Long, val underlyingSplit: Any) { +class SplitInfo( + val inputFormatClazz: Class[_], + val hostLocation: String, + val path: String, + val length: Long, + val underlyingSplit: Any) { override def toString(): String = { "SplitInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", hostLocation : " + hostLocation + ", path : " + path + diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 9f732f7191465..b42e231e11f91 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.RDDInfo * Stores information about a stage to pass from the scheduler to SparkListeners. */ @DeveloperApi -class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { +class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfos: Seq[RDDInfo]) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None /** Time when all tasks in the stage completed or when the stage was cancelled. */ @@ -41,12 +41,17 @@ class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddIn } } -private[spark] -object StageInfo { +private[spark] object StageInfo { + /** + * Construct a StageInfo from a Stage. + * + * Each Stage is associated with one or many RDDs, with the boundary of a Stage marked by + * shuffle dependencies. Therefore, all ancestor RDDs related to this Stage's RDD through a + * sequence of narrow dependencies should also be associated with this Stage. + */ def fromStage(stage: Stage): StageInfo = { - val rdd = stage.rdd - val rddName = Option(rdd.name).getOrElse(rdd.id.toString) - val rddInfo = new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) - new StageInfo(stage.id, stage.name, stage.numTasks, rddInfo) + val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) + val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos + new StageInfo(stage.id, stage.name, stage.numTasks, rddInfos) } } 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 a8bcb7dfe2f3c..2ca3479c80efc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -44,8 +44,9 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex final def run(attemptId: Long): T = { context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false) + taskThread = Thread.currentThread() if (_killed) { - kill() + kill(interruptThread = false) } runTask(context) } @@ -62,6 +63,9 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex // Task context, to be initialized in run(). @transient protected var context: TaskContext = _ + // The actual Thread on which the task is running, if any. Initialized in run(). + @volatile @transient private var taskThread: Thread = _ + // A flag to indicate whether the task is killed. This is used in case context is not yet // initialized when kill() is invoked. @volatile @transient private var _killed = false @@ -75,12 +79,16 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex * Kills a task by setting the interrupted flag to true. This relies on the upper level Spark * code and user code to properly handle the flag. This function should be idempotent so it can * be called multiple times. + * If interruptThread is true, we will also call Thread.interrupt() on the Task's executor thread. */ - def kill() { + def kill(interruptThread: Boolean) { _killed = true if (context != null) { context.interrupted = true } + if (interruptThread && taskThread != null) { + taskThread.interrupt() + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 92616c997e20c..819c35257b5a7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -47,7 +47,7 @@ private[spark] trait TaskScheduler { def submitTasks(taskSet: TaskSet): Unit // Cancel a stage. - def cancelTasks(stageId: Int) + def cancelTasks(stageId: Int, interruptThread: Boolean) // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. def setDAGScheduler(dagScheduler: DAGScheduler): Unit 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 fe72ab3e43146..5a68f38bc5844 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -170,7 +170,7 @@ private[spark] class TaskSchedulerImpl( backend.reviveOffers() } - override def cancelTasks(stageId: Int): Unit = synchronized { + override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { logInfo("Cancelling stage " + stageId) activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => // There are two possible cases here: @@ -181,7 +181,7 @@ private[spark] class TaskSchedulerImpl( // simply abort the stage. tsm.runningTasksSet.foreach { tid => val execId = taskIdToExecutorId(tid) - backend.killTask(tid, execId) + backend.killTask(tid, execId, interruptThread) } tsm.abort("Stage %s cancelled".format(stageId)) logInfo("Stage %d was cancelled".format(stageId)) @@ -356,6 +356,7 @@ private[spark] class TaskSchedulerImpl( if (taskResultGetter != null) { taskResultGetter.stop() } + starvationTimer.cancel() // sleeping for an arbitrary 1 seconds to ensure that messages are sent out. Thread.sleep(1000L) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 03bf76083761f..613fa7850bb25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -31,8 +31,8 @@ private[spark] class TaskSet( val properties: Properties) { val id: String = stageId + "." + attempt - def kill() { - tasks.foreach(_.kill()) + def kill(interruptThread: Boolean) { + tasks.foreach(_.kill(interruptThread)) } override def toString: String = "TaskSet " + id 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 a81b834267f0d..f3bd0797aa035 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -465,7 +465,7 @@ private[spark] class TaskSetManager( def handleTaskGettingResult(tid: Long) = { val info = taskInfos(tid) info.markGettingResult() - sched.dagScheduler.taskGettingResult(tasks(info.index), info) + sched.dagScheduler.taskGettingResult(info) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 4a9a1659d8254..ddbc74e82ac49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -30,7 +30,8 @@ private[spark] object CoarseGrainedClusterMessages { // Driver to executors case class LaunchTask(task: TaskDescription) extends CoarseGrainedClusterMessage - case class KillTask(taskId: Long, executor: String) extends CoarseGrainedClusterMessage + case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) + extends CoarseGrainedClusterMessage case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) extends CoarseGrainedClusterMessage 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 7bfc30b4208a3..a6d6b3d26a3c6 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 @@ -101,8 +101,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case ReviveOffers => makeOffers() - case KillTask(taskId, executorId) => - executorActor(executorId) ! KillTask(taskId, executorId) + case KillTask(taskId, executorId, interruptThread) => + executorActor(executorId) ! KillTask(taskId, executorId, interruptThread) case StopDriver => sender ! true @@ -207,8 +207,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A driverActor ! ReviveOffers } - override def killTask(taskId: Long, executorId: String) { - driverActor ! KillTask(taskId, executorId) + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { + driverActor ! KillTask(taskId, executorId, interruptThread) } override def defaultParallelism(): Int = { 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 936e9db80573d..9544ca05dca70 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 @@ -42,11 +42,20 @@ private[spark] class SparkDeploySchedulerBackend( // The endpoint for executors to talk to us val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), conf.get("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + val classPathEntries = sys.props.get("spark.executor.extraClassPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val libraryPathEntries = sys.props.get("spark.executor.extraLibraryPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) + "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.testExecutorEnvs, + classPathEntries, libraryPathEntries, extraJavaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 06b041e1fd9a9..2cd9d6c12eaf7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -111,7 +111,18 @@ private[spark] class CoarseMesosSchedulerBackend( def createCommand(offer: Offer, numCores: Int): CommandInfo = { val environment = Environment.newBuilder() - sc.executorEnvs.foreach { case (key, value) => + val extraClassPath = conf.getOption("spark.executor.extraClassPath") + extraClassPath.foreach { cp => + environment.addVariables( + Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) + } + val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions") + + val libraryPathOption = "spark.executor.extraLibraryPath" + val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") + val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") + + sc.testExecutorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) @@ -123,20 +134,22 @@ private[spark] class CoarseMesosSchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) + val uri = conf.get("spark.executor.uri", null) if (uri == null) { val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath command.setValue( - "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d".format( + runScript, extraOpts, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( ("cd %s*; " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d") - .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d") + .format(basename, extraOpts, driverUrl, offer.getSlaveId.getValue, + offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } command.build() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index dfdcafe19fb93..c975f312324ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -90,7 +90,7 @@ private[spark] class MesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) val environment = Environment.newBuilder() - sc.executorEnvs.foreach { case (key, value) => + sc.testExecutorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) 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 16e2f5cf3076d..43f0e18a0cbe0 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 @@ -30,7 +30,7 @@ private case class ReviveOffers() private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) -private case class KillTask(taskId: Long) +private case class KillTask(taskId: Long, interruptThread: Boolean) /** * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on @@ -61,8 +61,8 @@ private[spark] class LocalActor( reviveOffers() } - case KillTask(taskId) => - executor.killTask(taskId) + case KillTask(taskId, interruptThread) => + executor.killTask(taskId, interruptThread) } def reviveOffers() { @@ -99,8 +99,8 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: override def defaultParallelism() = totalCores - override def killTask(taskId: Long, executorId: String) { - localActor ! KillTask(taskId) + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { + localActor ! KillTask(taskId, interruptThread) } override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { 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 d1e8c3ef63622..c4daec7875d26 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -58,7 +58,9 @@ class KryoSerializer(conf: SparkConf) // Do this before we invoke the user registrator so the user registrator can override this. kryo.setReferences(referenceTracking) - for (cls <- KryoSerializer.toRegister) kryo.register(cls) + for (cls <- KryoSerializer.toRegister) { + kryo.register(cls) + } // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) @@ -77,7 +79,7 @@ class KryoSerializer(conf: SparkConf) } // Register Chill's classes; we do this after our ranges and the user's own classes to let - // our code override the generic serialziers in Chill for things like Seq + // our code override the generic serializers in Chill for things like Seq new AllScalaRegistrar().apply(kryo) kryo.setClassLoader(classLoader) @@ -176,10 +178,6 @@ private[serializer] object KryoSerializer { classOf[GetBlock], classOf[MapStatus], classOf[BlockManagerId], - classOf[Array[Byte]], - (1 to 10).getClass, - (1 until 10).getClass, - (1L to 10L).getClass, - (1L until 10L).getClass + classOf[Array[Byte]] ) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index ace9cd51c96b7..a02dd9441d679 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -148,6 +148,12 @@ object BlockFetcherIterator { } protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] @@ -159,11 +165,6 @@ object BlockFetcherIterator { _numBlocksToFetch += localBlocksToFetch.size } else { numRemote += blockInfos.size - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) val iterator = blockInfos.iterator var curRequestSize = 0L var curBlocks = new ArrayBuffer[(BlockId, Long)] @@ -178,11 +179,12 @@ object BlockFetcherIterator { } else if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } - if (curRequestSize >= minRequestSize) { + if (curRequestSize >= targetRequestSize) { // Add this FetchRequest remoteRequests += new FetchRequest(address, curBlocks) curRequestSize = 0 curBlocks = new ArrayBuffer[(BlockId, Long)] + logDebug(s"Creating fetch request of $curRequestSize at $address") } } // Add in the final request @@ -191,7 +193,7 @@ object BlockFetcherIterator { } } } - logInfo("Getting " + _numBlocksToFetch + " non-zero-bytes blocks out of " + + logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + totalBlocks + " blocks") remoteRequests } @@ -226,8 +228,8 @@ object BlockFetcherIterator { sendRequest(fetchRequests.dequeue()) } - val numGets = remoteRequests.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + val numFetches = remoteRequests.size - fetchRequests.size + logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) // Get Local Blocks startTime = System.currentTimeMillis @@ -327,7 +329,7 @@ object BlockFetcherIterator { } copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6)) - logInfo("Started " + fetchRequestsSync.size + " remote gets in " + + logInfo("Started " + fetchRequestsSync.size + " remote fetches in " + Utils.getUsedTimeMs(startTime)) // Get Local Blocks diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index f15fa4dd7ffd5..6d7d4f922e1fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -46,11 +46,12 @@ private[spark] class BlockManager( val master: BlockManagerMaster, val defaultSerializer: Serializer, maxMemory: Long, - val conf: SparkConf, + val _conf: SparkConf, securityManager: SecurityManager, mapOutputTracker: MapOutputTracker) extends Logging { + def conf = _conf val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) @@ -281,7 +282,9 @@ private[spark] class BlockManager( val onDiskSize = status.diskSize master.updateBlockInfo( blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inTachyonSize) - } else true + } else { + true + } } /** @@ -676,7 +679,7 @@ private[spark] class BlockManager( tachyonStore.putValues(blockId, iterator, level, false) case ArrayBufferValues(array) => tachyonStore.putValues(blockId, array, level, false) - case ByteBufferValues(bytes) => + case ByteBufferValues(bytes) => bytes.rewind() tachyonStore.putBytes(blockId, bytes, level) } @@ -695,7 +698,7 @@ private[spark] class BlockManager( diskStore.putValues(blockId, iterator, level, askForBytes) case ArrayBufferValues(array) => diskStore.putValues(blockId, array, level, askForBytes) - case ByteBufferValues(bytes) => + case ByteBufferValues(bytes) => bytes.rewind() diskStore.putBytes(blockId, bytes, level) } @@ -1021,6 +1024,8 @@ private[spark] class BlockManager( heartBeatTask.cancel() } connectionManager.stop() + shuffleBlockManager.stop() + diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() 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 c57b6e8391b13..63fa5d3eb6541 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -203,7 +203,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val locations = blockLocations.get(blockId) locations -= blockManagerId if (locations.size == 0) { - blockLocations.remove(locations) + blockLocations.remove(blockId) } } listenerBus.post(SparkListenerBlockManagerRemoved(blockManagerId)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala index 337b45b727dec..a2bfce7b4a0fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala @@ -57,7 +57,6 @@ private[spark] class BlockMessage() { } def set(buffer: ByteBuffer) { - val startTime = System.currentTimeMillis /* println() println("BlockMessage: ") @@ -100,7 +99,6 @@ private[spark] class BlockMessage() { data.flip() } - val finishTime = System.currentTimeMillis } def set(bufferMsg: BufferMessage) { @@ -115,7 +113,6 @@ private[spark] class BlockMessage() { def getLevel: StorageLevel = level def toBufferMessage: BufferMessage = { - val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() var buffer = ByteBuffer.allocate(4 + 4 + id.name.length * 2) buffer.putInt(typ).putInt(id.name.length) @@ -153,7 +150,6 @@ private[spark] class BlockMessage() { println() println() */ - val finishTime = System.currentTimeMillis Message.createBufferMessage(buffers) } 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 7a24c8f57f43b..cf6ef0029a861 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -150,20 +150,27 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") - localDirs.foreach { localDir => - try { - if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) - } catch { - case t: Throwable => - logError("Exception while deleting local spark dir: " + localDir, t) - } - } + DiskBlockManager.this.stop() + } + }) + } - if (shuffleSender != null) { - shuffleSender.stop() + /** Cleanup local dirs and stop shuffle sender. */ + private[spark] def stop() { + localDirs.foreach { localDir => + if (localDir.isDirectory() && localDir.exists()) { + try { + if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) + } catch { + case t: Throwable => + logError("Exception while deleting local spark dir: " + localDir, t) } } - }) + } + + if (shuffleSender != null) { + shuffleSender.stop() + } } private[storage] def startShuffleBlockSender(port: Int): Int = { diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 36ee4bcc41c66..0ab9fad422717 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -33,6 +33,8 @@ import org.apache.spark.util.Utils private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { + val minMemoryMapBytes = blockManager.conf.getLong("spark.storage.memoryMapThreshold", 2 * 4096L) + override def getSize(blockId: BlockId): Long = { diskManager.getBlockLocation(blockId).length } @@ -94,12 +96,20 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage override def getBytes(blockId: BlockId): Option[ByteBuffer] = { val segment = diskManager.getBlockLocation(blockId) val channel = new RandomAccessFile(segment.file, "r").getChannel() - val buffer = try { - channel.map(MapMode.READ_ONLY, segment.offset, segment.length) + + try { + // For small files, directly read rather than memory map + if (segment.length < minMemoryMapBytes) { + val buf = ByteBuffer.allocate(segment.length.toInt) + channel.read(buf, segment.offset) + buf.flip() + Some(buf) + } else { + Some(channel.map(MapMode.READ_ONLY, segment.offset, segment.length)) + } } finally { channel.close() } - Some(buffer) } override def getValues(blockId: BlockId): Option[Iterator[Any]] = { diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala new file mode 100644 index 0000000000000..023fd6e4d8baa --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -0,0 +1,55 @@ +/* + * 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.storage + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +@DeveloperApi +class RDDInfo( + val id: Int, + val name: String, + val numPartitions: Int, + val storageLevel: StorageLevel) + extends Ordered[RDDInfo] { + + var numCachedPartitions = 0 + var memSize = 0L + var diskSize = 0L + var tachyonSize = 0L + + override def toString = { + import Utils.bytesToString + ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + + "TachyonSize: %s; DiskSize: %s").format( + name, id, storageLevel.toString, numCachedPartitions, numPartitions, + bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) + } + + override def compare(that: RDDInfo) = { + this.id - that.id + } +} + +private[spark] object RDDInfo { + def fromRdd(rdd: RDD[_]): RDDInfo = { + val rddName = Option(rdd.name).getOrElse(rdd.id.toString) + new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 4cd4cdbd9909d..35910e552fe86 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -207,6 +207,10 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { private def cleanup(cleanupTime: Long) { shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId)) } + + def stop() { + metadataCleaner.cancel() + } } private[spark] diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 95e71de2d3f1d..c9a52e0366d93 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -19,6 +19,8 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import org.apache.spark.annotation.DeveloperApi + /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to @@ -142,21 +144,37 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) - /** Create a new StorageLevel object without setting useOffHeap */ + /** + * :: DeveloperApi :: + * Create a new StorageLevel object without setting useOffHeap + */ + @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, deserialized: Boolean, replication: Int) = getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) - /** Create a new StorageLevel object */ + /** + * :: DeveloperApi :: + * Create a new StorageLevel object + */ + @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( new StorageLevel(useDisk, useMemory, false, deserialized, replication)) - /** Create a new StorageLevel object from its integer representation */ + /** + * :: DeveloperApi :: + * Create a new StorageLevel object from its integer representation + */ + @DeveloperApi def apply(flags: Int, replication: Int): StorageLevel = getCachedStorageLevel(new StorageLevel(flags, replication)) - /** Read StorageLevel object from ObjectInput stream */ + /** + * :: DeveloperApi :: + * Read StorageLevel object from ObjectInput stream + */ + @DeveloperApi def apply(in: ObjectInput): StorageLevel = { val obj = new StorageLevel() obj.readExternal(in) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 7ed371326855d..1eddd1cdc483b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,60 +21,30 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.Utils -private[spark] -class StorageStatus( +/** Storage information for each BlockManager. */ +private[spark] class StorageStatus( val blockManagerId: BlockManagerId, val maxMem: Long, val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { - def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + def memUsed = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) def memUsedByRDD(rddId: Int) = rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + def diskUsed = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) def diskUsedByRDD(rddId: Int) = rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) - def memRemaining : Long = maxMem - memUsed() + def memRemaining: Long = maxMem - memUsed def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } } -@DeveloperApi -private[spark] -class RDDInfo( - val id: Int, - val name: String, - val numPartitions: Int, - val storageLevel: StorageLevel) - extends Ordered[RDDInfo] { - - var numCachedPartitions = 0 - var memSize = 0L - var diskSize = 0L - var tachyonSize = 0L - - override def toString = { - import Utils.bytesToString - ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s;" + - "TachyonSize: %s; DiskSize: %s").format( - name, id, storageLevel.toString, numCachedPartitions, numPartitions, - bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) - } - - override def compare(that: RDDInfo) = { - this.id - that.id - } -} - -/* Helper methods for storage-related objects */ -private[spark] -object StorageUtils { +/** Helper methods for storage-related objects. */ +private[spark] object StorageUtils { /** * Returns basic information of all RDDs persisted in the given SparkContext. This does not diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 750f5a501c213..b3ac2320f3431 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -137,8 +137,8 @@ private[spark] object JettyUtils extends Logging { contextHandler } - /** Add security filters, if any, do the given list of ServletContextHandlers */ - private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { + /** Add filters, if any, to the given list of ServletContextHandlers */ + def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) filters.foreach { case filter : String => @@ -195,6 +195,7 @@ private[spark] object JettyUtils extends Logging { (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() + pool.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) logInfo("Error was: " + f.toString) connect((currentPort + 1) % 65536) 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 2fef1a635427c..097a1b81e1dd1 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -43,6 +43,14 @@ private[spark] class SparkUI( def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = this(null, conf, new SecurityManager(conf), listenerBus, appName, basePath) + def this( + conf: SparkConf, + securityManager: SecurityManager, + listenerBus: SparkListenerBus, + appName: String, + basePath: String) = + this(null, conf, securityManager, listenerBus, appName, basePath) + // If SparkContext is not provided, assume the associated application is not live val live = sc != null 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 99770f28549c9..a3d6a1821245b 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -121,7 +121,11 @@ private[spark] object UIUtils extends Logging { (records, "") } } - "%.1f%s".formatLocal(Locale.US, value, unit) + if (unit.isEmpty) { + "%d".formatLocal(Locale.US, value.toInt) + } else { + "%.1f%s".formatLocal(Locale.US, value, unit) + } } // Yarn has to go through a proxy so the base uri is provided and has to be on all links diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index c1e69f6cdaffb..6cb43c02b8f08 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -32,7 +32,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) - val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) + val memUsed = storageStatusList.map(_.memUsed).fold(0L)(_ + _) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.getOrElse("Executor ID", "")) @@ -106,9 +106,9 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort val rddBlocks = status.blocks.size - val memUsed = status.memUsed() + val memUsed = status.memUsed val maxMem = status.maxMem - val diskUsed = status.diskUsed() + val diskUsed = status.diskUsed val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0) val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 34ff2ac34a7ca..0da62892118d4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -41,10 +41,12 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") val now = System.currentTimeMillis val activeStagesTable = - new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent, parent.killEnabled) val completedStagesTable = - new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) - val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent) + val failedStagesTable = + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs val pools = if (live) sc.getAllPools else Seq[Schedulable]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index fd83d37583967..0a2bf31833d2b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -40,7 +40,8 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { case Some(s) => s.values.toSeq case None => Seq[StageInfo]() } - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) + val activeStagesTable = + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq[Schedulable]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 8c5b1f55fd2dc..153434a2032be 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -27,34 +27,36 @@ import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[ui] class StageTable( +private[ui] class StageTableBase( stages: Seq[StageInfo], parent: JobProgressTab, killEnabled: Boolean = false) { private val basePath = parent.basePath private val listener = parent.listener - private lazy val isFairScheduler = parent.isFairScheduler + protected def isFairScheduler = parent.isFairScheduler + + protected def columns: Seq[Node] = { + Stage Id ++ + {if (isFairScheduler) {Pool Name} else Seq.empty} ++ + Description + Submitted + Duration + Tasks: Succeeded/Total + Shuffle Read + Shuffle Write + } def toNodeSeq: Seq[Node] = { listener.synchronized { - stageTable(stageRow, stages) + stageTable(renderStageRow, stages) } } /** Special table that merges two header cells. */ - private def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + protected def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - - {if (isFairScheduler) {} else {}} - - - - - - - + {columns} {rows.map(r => makeRow(r))} @@ -94,8 +96,7 @@ private[ui] class StageTable( .getOrElse(
{killLink}{nameLink}
) } - /** Render an HTML row that represents a stage */ - private def stageRow(s: StageInfo): Seq[Node] = { + protected def stageRow(s: StageInfo): Seq[Node] = { val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { case Some(t) => UIUtils.formatDate(new Date(t)) @@ -124,25 +125,42 @@ private[ui] class StageTable( case 0 => "" case b => Utils.bytesToString(b) } - - - - {if (isFairScheduler) { - - }} - - - - ++ + {if (isFairScheduler) { + - - - + } else { + Seq.empty + }} ++ + + + + + + + } + + /** Render an HTML row that represents a stage */ + private def renderStageRow(s: StageInfo): Seq[Node] = {stageRow(s)} +} + +private[ui] class FailedStageTable( + stages: Seq[StageInfo], + parent: JobProgressTab, + killEnabled: Boolean = false) + extends StageTableBase(stages, parent, killEnabled) { + + override protected def columns: Seq[Node] = super.columns ++ + + override protected def stageRow(s: StageInfo): Seq[Node] = { + val basicColumns = super.stageRow(s) + val failureReason = + basicColumns ++ failureReason } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index d07f1c9b20fcf..2155633b8096f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -26,7 +26,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RddPage(parent: StorageTab) extends WebUIPage("rdd") { +private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { private val appName = parent.appName private val basePath = parent.basePath private val listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 56429f6c07fcd..07ec297841f20 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -30,7 +30,7 @@ private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage" val listener = new StorageListener(parent.storageStatusListener) attachPage(new StoragePage(this)) - attachPage(new RddPage(this)) + attachPage(new RDDPage(this)) parent.registerListener(listener) } @@ -66,8 +66,8 @@ private[ui] class StorageListener(storageStatusListener: StorageStatusListener) } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { - val rddInfo = stageSubmitted.stageInfo.rddInfo - _rddInfoMap.getOrElseUpdate(rddInfo.id, rddInfo) + val rddInfos = stageSubmitted.stageInfo.rddInfos + rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index b9f4a5d720b93..1b2b1932e0c3d 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -43,8 +43,11 @@ private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Orderin } override def +=(elem: A): this.type = { - if (size < maxSize) underlying.offer(elem) - else maybeReplaceLowest(elem) + if (size < maxSize) { + underlying.offer(elem) + } else { + maybeReplaceLowest(elem) + } this } @@ -59,7 +62,8 @@ private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Orderin if (head != null && ord.gt(a, head)) { underlying.poll() underlying.offer(a) - } else false + } else { + false + } } } - diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 68a12e8ed67d7..0965e0f0f7828 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -22,7 +22,9 @@ import java.net.URI import java.text.SimpleDateFormat import java.util.Date -import org.apache.hadoop.fs.{FSDataOutputStream, Path} +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.io.CompressionCodec @@ -37,10 +39,12 @@ import org.apache.spark.io.CompressionCodec */ private[spark] class FileLogger( logDir: String, - conf: SparkConf = new SparkConf, + conf: SparkConf, + hadoopConfiguration: Configuration, outputBufferSize: Int = 8 * 1024, // 8 KB compress: Boolean = false, - overwrite: Boolean = true) + overwrite: Boolean = true, + dirPermissions: Option[FsPermission] = None) extends Logging { private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -77,28 +81,38 @@ private[spark] class FileLogger( 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): PrintWriter = { + private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = { val logPath = logDir + "/" + fileName val uri = new URI(logPath) + val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme + val isDefaultLocal = (defaultFs == null || defaultFs == "file") + val path = new Path(logPath) /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ - val dstream = uri.getScheme match { - case "file" | null => + val dstream = + if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { // Second parameter is whether to append - new FileOutputStream(logPath, !overwrite) - - case _ => - val path = new Path(logPath) + 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) @@ -110,7 +124,9 @@ private[spark] class FileLogger( * @param withTime Whether to prepend message with a timestamp */ def log(msg: String, withTime: Boolean = false) { - val writeInfo = if (!withTime) msg else { + val writeInfo = if (!withTime) { + msg + } else { val date = new Date(System.currentTimeMillis()) dateFormat.get.format(date) + ": " + msg } @@ -147,15 +163,16 @@ private[spark] class FileLogger( /** * 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 = "") { + 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)) + writer = Some(createWriter(name, perms)) } /** 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 465835ea7fe29..9aed3e0985654 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -176,7 +176,7 @@ private[spark] object JsonProtocol { * -------------------------------------------------------------------- */ def stageInfoToJson(stageInfo: StageInfo): JValue = { - val rddInfo = rddInfoToJson(stageInfo.rddInfo) + val rddInfo = JArray(stageInfo.rddInfos.map(rddInfoToJson).toList) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing) @@ -208,7 +208,8 @@ private[spark] object JsonProtocol { taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) val shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) - val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => + val updatedBlocks = + taskMetrics.updatedBlocks.map { blocks => JArray(blocks.toList.map { case (id, status) => ("Block ID" -> id.toString) ~ ("Status" -> blockStatusToJson(status)) @@ -467,13 +468,13 @@ private[spark] object JsonProtocol { val stageId = (json \ "Stage ID").extract[Int] val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] - val rddInfo = rddInfoFromJson(json \ "RDD Info") + val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson) val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] - val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) + val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfos) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason @@ -518,13 +519,14 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) - metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => - value.extract[List[JValue]].map { block => - val id = BlockId((block \ "Block ID").extract[String]) - val status = blockStatusFromJson(block \ "Status") - (id, status) + metrics.updatedBlocks = + Utils.jsonOption(json \ "Updated Blocks").map { value => + value.extract[List[JValue]].map { block => + val id = BlockId((block \ "Block ID").extract[String]) + val status = blockStatusFromJson(block \ "Status") + (id, status) + } } - } metrics } 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 a3af4e7b91692..79f314c8dd36c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -28,8 +28,10 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag +import scala.util.Try import com.google.common.io.Files +import org.apache.commons.lang.SystemUtils import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ @@ -45,7 +47,12 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, */ private[spark] object Utils extends Logging { - val osName = System.getProperty("os.name") + val random = new Random() + + def sparkBin(sparkHome: String, which: String): File = { + val suffix = if (SystemUtils.IS_OS_WINDOWS) ".cmd" else "" + new File(sparkHome + File.separator + "bin", which + suffix) + } /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { @@ -131,6 +138,11 @@ private[spark] object Utils extends Logging { def getContextOrSparkClassLoader = Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader) + /** Determines whether the provided class is loadable in the current thread. */ + def classIsLoadable(clazz: String): Boolean = { + Try { Class.forName(clazz, false, getContextOrSparkClassLoader) }.isSuccess + } + /** * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. */ @@ -547,8 +559,7 @@ private[spark] object Utils extends Logging { } /** - * Return the string to tell how long has passed in seconds. The passing parameter should be in - * millisecond. + * Return the string to tell how long has passed in milliseconds. */ def getUsedTimeMs(startTimeMs: Long): String = { " " + (System.currentTimeMillis - startTimeMs) + " ms" @@ -603,7 +614,7 @@ private[spark] object Utils extends Logging { */ def isSymlink(file: File): Boolean = { if (file == null) throw new NullPointerException("File must not be null") - if (osName.startsWith("Windows")) return false + if (SystemUtils.IS_OS_WINDOWS) return false val fileInCanonicalDir = if (file.getParent() == null) { file } else { @@ -811,8 +822,7 @@ private[spark] object Utils extends Logging { } else { el.getMethodName } - } - else { + } else { firstUserLine = el.getLineNumber firstUserFile = el.getFileName firstUserClass = el.getClassName @@ -1007,10 +1017,18 @@ private[spark] object Utils extends Logging { if (dst.isAbsolute()) { throw new IOException("Destination must be relative") } - val linkCmd = if (osName.startsWith("Windows")) "copy" else "ln -sf" + var cmdSuffix = "" + val linkCmd = if (SystemUtils.IS_OS_WINDOWS) { + // refer to http://technet.microsoft.com/en-us/library/cc771254.aspx + cmdSuffix = " /s /e /k /h /y /i" + "cmd /c xcopy " + } else { + cmdSuffix = "" + "ln -sf " + } import scala.sys.process._ - (linkCmd + " " + src.getAbsolutePath() + " " + dst.getPath()) lines_! ProcessLogger(line => - (logInfo(line))) + (linkCmd + src.getAbsolutePath() + " " + dst.getPath() + cmdSuffix) lines_! + ProcessLogger(line => (logInfo(line))) } @@ -1043,4 +1061,11 @@ private[spark] object Utils extends Logging { def getHadoopFileSystem(path: String): FileSystem = { getHadoopFileSystem(new URI(path)) } + + /** + * Indicates whether Spark is currently running unit tests. + */ + private[spark] def isTesting = { + sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index ad38250ad339f..1a6f1c2b55799 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -40,6 +40,8 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") + private val LOAD_FACTOR = 0.7 + private var capacity = nextPowerOf2(initialCapacity) private var mask = capacity - 1 private var curSize = 0 @@ -57,8 +59,6 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) private var destroyed = false private val destructionMessage = "Map state is invalid from destructive sorting!" - private val LOAD_FACTOR = 0.7 - /** Get the value for a given key */ def apply(key: K): V = { assert(!destroyed, destructionMessage) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index d615767284c0b..170f09be21534 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -337,8 +337,8 @@ class ExternalAppendOnlyMap[K, V, C]( } override def compareTo(other: StreamBuffer): Int = { - // minus sign because mutable.PriorityQueue dequeues the max, not the min - -minKeyHash.compareTo(other.minKeyHash) + // descending order because mutable.PriorityQueue dequeues the max, not the min + if (other.minKeyHash < minKeyHash) -1 else if (other.minKeyHash == minKeyHash) 0 else 1 } } } @@ -422,7 +422,9 @@ class ExternalAppendOnlyMap[K, V, C]( private[spark] object ExternalAppendOnlyMap { private class KCComparator[K, C] extends Comparator[(K, C)] { def compare(kc1: (K, C), kc2: (K, C)): Int = { - kc1._1.hashCode().compareTo(kc2._1.hashCode()) + val hash1 = kc1._1.hashCode() + val hash2 = kc2._1.hashCode() + if (hash1 < hash2) -1 else if (hash1 == hash2) 0 else 1 } } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 8d2e9f1846343..76c6f5af82a5e 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -610,8 +610,8 @@ public void textFiles() throws IOException { @Test public void wholeTextFiles() throws IOException { - byte[] content1 = "spark is easy to use.\n".getBytes(); - byte[] content2 = "spark is also easy to use.\n".getBytes(); + byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); + byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); File tempDir = Files.createTempDir(); String tempDirName = tempDir.getAbsolutePath(); diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index b86923f07f793..fd5b0906e6765 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -60,7 +60,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -73,7 +73,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -87,7 +87,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala new file mode 100644 index 0000000000000..4bd889135631b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.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 + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ + +class ImplicitOrderingSuite extends FunSuite with LocalSparkContext { + class NonOrderedClass {} + + class ComparableClass extends Comparable[ComparableClass] { + override def compareTo(o: ComparableClass): Int = ??? + } + + class OrderedClass extends Ordered[OrderedClass] { + override def compare(o: OrderedClass): Int = ??? + } + + // Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should. + test("basic inference of Orderings"){ + sc = new SparkContext("local", "test") + val rdd = sc.parallelize(1 to 10) + + // Infer orderings after basic maps to particular types + assert(rdd.map(x => (x, x)).keyOrdering.isDefined) + assert(rdd.map(x => (1, x)).keyOrdering.isDefined) + assert(rdd.map(x => (x.toString, x)).keyOrdering.isDefined) + assert(rdd.map(x => (null, x)).keyOrdering.isDefined) + assert(rdd.map(x => (new NonOrderedClass, x)).keyOrdering.isEmpty) + assert(rdd.map(x => (new ComparableClass, x)).keyOrdering.isDefined) + assert(rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined) + + // Infer orderings for other RDD methods + assert(rdd.groupBy(x => x).keyOrdering.isDefined) + assert(rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty) + assert(rdd.groupBy(x => new ComparableClass).keyOrdering.isDefined) + assert(rdd.groupBy(x => new OrderedClass).keyOrdering.isDefined) + assert(rdd.groupBy((x: Int) => x, 5).keyOrdering.isDefined) + assert(rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined) + } +} diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 7a39d1af9e2d5..2c8ef405c944c 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -21,7 +21,7 @@ import java.util.concurrent.Semaphore import scala.concurrent.Await import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.duration.Duration +import scala.concurrent.duration._ import scala.concurrent.future import org.scalatest.{BeforeAndAfter, FunSuite} @@ -84,6 +84,35 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf assert(sc.parallelize(1 to 10, 2).count === 10) } + test("do not put partially executed partitions into cache") { + // In this test case, we create a scenario in which a partition is only partially executed, + // and make sure CacheManager does not put that partially executed partition into the + // BlockManager. + import JobCancellationSuite._ + sc = new SparkContext("local", "test") + + // Run from 1 to 10, and then block and wait for the task to be killed. + val rdd = sc.parallelize(1 to 1000, 2).map { x => + if (x > 10) { + taskStartedSemaphore.release() + taskCancelledSemaphore.acquire() + } + x + }.cache() + + val rdd1 = rdd.map(x => x) + + future { + taskStartedSemaphore.acquire() + sc.cancelAllJobs() + taskCancelledSemaphore.release(100000) + } + + intercept[SparkException] { rdd1.count() } + // If the partial block is put into cache, rdd.count() would return a number less than 1000. + assert(rdd.count() === 1000) + } + test("job group") { sc = new SparkContext("local[2]", "test") @@ -101,27 +130,57 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count() } + // Block until both tasks of job A have started and cancel job A. + sem.acquire(2) + sc.clearJobGroup() val jobB = sc.parallelize(1 to 100, 2).countAsync() + sc.cancelJobGroup("jobA") + val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + assert(e.getMessage contains "cancel") + + // Once A is cancelled, job B should finish fairly quickly. + assert(jobB.get() === 100) + } + + test("job group with interruption") { + sc = new SparkContext("local[2]", "test") + + // Add a listener to release the semaphore once any tasks are launched. + val sem = new Semaphore(0) + sc.addSparkListener(new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart) { + sem.release() + } + }) + + // jobA is the one to be cancelled. + val jobA = future { + sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true) + sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(100000); i }.count() + } // Block until both tasks of job A have started and cancel job A. sem.acquire(2) + + sc.clearJobGroup() + val jobB = sc.parallelize(1 to 100, 2).countAsync() sc.cancelJobGroup("jobA") - val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + val e = intercept[SparkException] { Await.result(jobA, 5.seconds) } assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. assert(jobB.get() === 100) } -/* - test("two jobs sharing the same stage") { + + ignore("two jobs sharing the same stage") { // sem1: make sure cancel is issued after some tasks are launched // sem2: make sure the first stage is not finished until cancel is issued val sem1 = new Semaphore(0) val sem2 = new Semaphore(0) sc = new SparkContext("local[2]", "test") - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem1.release() } @@ -147,7 +206,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf intercept[SparkException] { f1.get() } intercept[SparkException] { f2.get() } } - */ + def testCount() { // Cancel before launching any tasks { @@ -206,3 +265,9 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf } } } + + +object JobCancellationSuite { + val taskStartedSemaphore = new Semaphore(0) + val taskCancelledSemaphore = new Semaphore(0) +} diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index dfe057515efa0..0bb6a6b09c5b5 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -179,7 +179,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala new file mode 100644 index 0000000000000..e39093e24d68a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -0,0 +1,63 @@ +/* + * 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 + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +class SecurityManagerSuite extends FunSuite { + + test("set security with conf") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.ui.view.acls", "user1,user2") + val securityManager = new SecurityManager(conf); + assert(securityManager.isAuthenticationEnabled() === true) + assert(securityManager.uiAclsEnabled() === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkUIViewPermissions("user3") === false) + } + + test("set security with api") { + val conf = new SparkConf + conf.set("spark.ui.view.acls", "user1,user2") + val securityManager = new SecurityManager(conf); + securityManager.setUIAcls(true) + assert(securityManager.uiAclsEnabled() === true) + securityManager.setUIAcls(false) + assert(securityManager.uiAclsEnabled() === false) + + // acls are off so doesn't matter what view acls set to + assert(securityManager.checkUIViewPermissions("user4") === true) + + securityManager.setUIAcls(true) + assert(securityManager.uiAclsEnabled() === true) + securityManager.setViewAcls(ArrayBuffer[String]("user5"), "user6,user7") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkUIViewPermissions("user7") === true) + assert(securityManager.checkUIViewPermissions("user8") === false) + assert(securityManager.checkUIViewPermissions(null) === true) + } +} + 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 9f2924c23b73c..bfae32dae0dc5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -87,7 +87,7 @@ class JsonProtocolSuite extends FunSuite { } def createAppDesc(): ApplicationDescription = { - val cmd = new Command("mainClass", List("arg1", "arg2"), Map()) + val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq()) new ApplicationDescription("name", Some(4), 1234, cmd, Some("sparkHome"), "appUiUrl") } @@ -100,7 +100,7 @@ class JsonProtocolSuite extends FunSuite { def createDriverCommand() = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), - Map(("K1", "V1"), ("K2", "V2")) + Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Some("-Dfoo") ) def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, @@ -133,9 +133,12 @@ class JsonProtocolSuite extends FunSuite { def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { val Diff(c, a, d) = validateJson diff expectedJson - assert(c === JNothing, "Json changed") - assert(a === JNothing, "Json added") - assert(d === JNothing, "Json deleted") + val validatePretty = JsonMethods.pretty(validateJson) + val expectedPretty = JsonMethods.pretty(expectedJson) + val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" + assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") + assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") + assert(d === JNothing, s"$errorMessage\nDelected:\n${JsonMethods.pretty(d)}") } } @@ -165,7 +168,7 @@ object JsonConstants { """ |{"name":"name","cores":4,"memoryperslave":1234, |"user":"%s","sparkhome":"sparkHome", - |"command":"Command(mainClass,List(arg1, arg2),Map())"} + |"command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),None)"} """.format(System.getProperty("user.name", "")).stripMargin val executorRunnerJsonStr = 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 4e489cd9b66a6..b3541b4a40b79 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -17,17 +17,20 @@ package org.apache.spark.deploy -import java.io.{OutputStream, PrintStream} +import java.io.{File, OutputStream, PrintStream} import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException, TestUtils} +import org.apache.spark.deploy.SparkSubmit._ +import org.apache.spark.util.Utils import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.deploy.SparkSubmit._ - - class SparkSubmitSuite extends FunSuite with ShouldMatchers { + def beforeAll() { + System.setProperty("spark.testing", "true") + } val noOpOutputStream = new OutputStream { def write(b: Int) = {} @@ -42,7 +45,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } /** Returns true if the script exits and the given search string is printed. */ - def testPrematureExit(input: Array[String], searchString: String): Boolean = { + def testPrematureExit(input: Array[String], searchString: String) = { val printStream = new BufferPrintStream() SparkSubmit.printStream = printStream @@ -60,37 +63,49 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } thread.start() thread.join() - printStream.lineBuffer.find(s => s.contains(searchString)).size > 0 + val joined = printStream.lineBuffer.mkString("\n") + if (!joined.contains(searchString)) { + fail(s"Search string '$searchString' not found in $joined") + } } test("prints usage on empty input") { - testPrematureExit(Array[String](), "Usage: spark-submit") should be (true) + testPrematureExit(Array[String](), "Usage: spark-submit") } test("prints usage with only --help") { - testPrematureExit(Array("--help"), "Usage: spark-submit") should be (true) + testPrematureExit(Array("--help"), "Usage: spark-submit") } - test("handles multiple binary definitions") { - val adjacentJars = Array("foo.jar", "bar.jar") - testPrematureExit(adjacentJars, "error: Found two conflicting resources") should be (true) - - val nonAdjacentJars = - Array("foo.jar", "--master", "123", "--class", "abc", "bar.jar") - testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") should be (true) + test("prints error with unrecognized options") { + testPrematureExit(Array("--blarg"), "Unrecognized option '--blarg'") + testPrematureExit(Array("-bleg"), "Unrecognized option '-bleg'") } test("handle binary specified but not class") { - testPrematureExit(Array("foo.jar"), "must specify a main class") + testPrematureExit(Array("foo.jar"), "Must specify a main class") + } + + test("handles arguments with --key=val") { + val clArgs = Seq("--jars=one.jar,two.jar,three.jar", "--name=myApp") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.jars should be ("one.jar,two.jar,three.jar") + appArgs.name should be ("myApp") + } + + test("handles arguments to user program") { + val clArgs = Seq("--name", "myApp", "userjar.jar", "some", "--random", "args", "here") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.childArgs should be (Seq("some", "--random", "args", "here")) } test("handles YARN cluster mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", + val clArgs = Seq("--deploy-mode", "cluster", "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", - "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", + "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -100,23 +115,23 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { childArgsStr should include ("--executor-memory 5g") childArgsStr should include ("--driver-memory 4g") childArgsStr should include ("--executor-cores 5") - childArgsStr should include ("--args arg1 --args arg2") + childArgsStr should include ("--arg arg1 --arg arg2") childArgsStr should include ("--queue thequeue") childArgsStr should include ("--files file1.txt,file2.txt") childArgsStr should include ("--archives archive1.txt,archive2.txt") childArgsStr should include ("--num-executors 6") mainClass should be ("org.apache.spark.deploy.yarn.Client") classpath should have length (0) - sysProps should have size (0) + sysProps should have size (1) } test("handles YARN client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", + val clArgs = Seq("--deploy-mode", "client", "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", - "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -131,28 +146,27 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { sysProps("spark.yarn.dist.files") should be ("file1.txt,file2.txt") sysProps("spark.yarn.dist.archives") should be ("archive1.txt,archive2.txt") sysProps("spark.executor.instances") should be ("6") + sysProps("SPARK_SUBMIT") should be ("true") } test("handles standalone cluster mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", - "--master", "spark://h:p", "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--supervise", "--driver-memory", "4g", "--driver-cores", "5") + val clArgs = Seq("--deploy-mode", "cluster", + "--master", "spark://h:p", "--class", "org.SomeClass", + "--supervise", "--driver-memory", "4g", "--driver-cores", "5", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") - print("child args: " + childArgsStr) childArgsStr.startsWith("--memory 4g --cores 5 --supervise") should be (true) childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") classpath should have length (0) - sysProps should have size (0) + sysProps should have size (2) // contains --jar entry and SPARK_SUBMIT } test("handles standalone client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", + val clArgs = Seq("--deploy-mode", "client", "--master", "spark://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--driver-memory", "4g") + "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -163,10 +177,9 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles mesos client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", + val clArgs = Seq("--deploy-mode", "client", "--master", "mesos://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--driver-memory", "4g") + "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -175,4 +188,82 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") } + + test("launch simple application with spark-submit") { + runSparkSubmit( + Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + "unUsed.jar")) + } + + test("spark submit includes jars passed in through --jar") { + val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) + val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) + val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") + val args = Seq( + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,512]", + "--jars", jarsString, + "unused.jar") + runSparkSubmit(args) + } + + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. + def runSparkSubmit(args: Seq[String]): String = { + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + Utils.executeAndGetOutput( + Seq("./bin/spark-submit") ++ args, + new File(sparkHome), + Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) + } +} + +object JarCreationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + val result = sc.makeRDD(1 to 100, 10).mapPartitions{ x => + var foundClasses = false + try { + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + foundClasses = true + } catch { + case _: Throwable => // catch all + } + Seq(foundClasses).iterator + }.collect() + if (result.contains(false)) { + throw new Exception("Could not load user defined classes inside of executors") + } + } +} + +object SimpleApplicationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + + val configs = Seq("spark.master", "spark.app.name") + for (config <- configs) { + val masterValue = conf.get(config) + val executorValues = sc + .makeRDD(1 to 100, 10) + .map(x => SparkEnv.get.conf.get(config)) + .collect() + .distinct + if (executorValues.size != 1) { + throw new SparkException(s"Inconsistent values for $config: $executorValues") + } + val executorValue = executorValues(0) + if (executorValue != masterValue) { + throw new SparkException( + s"Master had $config=$masterValue but executor had $config=$executorValue") + } + } + + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index a2c131b0c9787..4633bc3f7f25e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -29,7 +29,7 @@ import org.apache.spark.deploy.{Command, DriverDescription} class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { - val command = new Command("mainClass", Seq(), Map()) + val command = new Command("mainClass", Seq(), Map(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription, null, "akka://1.2.3.4/worker/") 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 3cab8e7b37934..8ae387fa0be6f 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 @@ -27,7 +27,8 @@ class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")) - val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(),Map()), + val appDesc = new ApplicationDescription("app name", Some(8), 500, + Command("foo", Seq(), Map(), Seq(), Seq()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome.getOrElse(".")), diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 1901330d8b188..8da9a0da700e0 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -17,12 +17,10 @@ package org.apache.spark.rdd -import scala.collection.mutable.HashMap -import scala.collection.parallel.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.reflect.ClassTag import org.scalatest.FunSuite -import org.scalatest.concurrent.Timeouts._ -import org.scalatest.time.{Millis, Span} import org.apache.spark._ import org.apache.spark.SparkContext._ @@ -153,7 +151,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { if (shouldFail) { throw new Exception("injected failure") } else { - return Array(1, 2, 3, 4).iterator + Array(1, 2, 3, 4).iterator } } }.cache() @@ -383,8 +381,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val prng42 = new Random(42) val prng43 = new Random(43) Array(1, 2, 3, 4, 5, 6).filter{i => - if (i < 4) 0 == prng42.nextInt(3) - else 0 == prng43.nextInt(3)} + if (i < 4) 0 == prng42.nextInt(3) else 0 == prng43.nextInt(3) + } } assert(sample.size === checkSample.size) for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) @@ -465,7 +463,13 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("takeSample") { val data = sc.parallelize(1 to 100, 2) - + + for (num <- List(5, 20, 100)) { + val sample = data.takeSample(withReplacement=false, num=num) + assert(sample.size === num) // Got exactly num elements + assert(sample.toSet.size === num) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) assert(sample.size === 20) // Got exactly 20 elements @@ -483,6 +487,19 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sample.size === 20) // Got exactly 20 elements assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") } + { + val sample = data.takeSample(withReplacement=true, num=20) + assert(sample.size === 20) // Got exactly 100 elements + assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + { + val sample = data.takeSample(withReplacement=true, num=100) + assert(sample.size === 100) // Got exactly 100 elements + // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=true, 100, seed) assert(sample.size === 100) // Got exactly 100 elements @@ -568,4 +585,144 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ids = ranked.map(_._1).distinct().collect() assert(ids.length === n) } + + test("getNarrowAncestors") { + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.filter(_ % 2 == 0).map(_ + 1) + val rdd3 = rdd2.map(_ - 1).filter(_ < 50).map(i => (i, i)) + val rdd4 = rdd3.reduceByKey(_ + _) + val rdd5 = rdd4.mapValues(_ + 1).mapValues(_ + 2).mapValues(_ + 3) + val ancestors1 = rdd1.getNarrowAncestors + val ancestors2 = rdd2.getNarrowAncestors + val ancestors3 = rdd3.getNarrowAncestors + val ancestors4 = rdd4.getNarrowAncestors + val ancestors5 = rdd5.getNarrowAncestors + + // Simple dependency tree with a single branch + assert(ancestors1.size === 0) + assert(ancestors2.size === 2) + assert(ancestors2.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) + assert(ancestors2.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors3.size === 5) + assert(ancestors3.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) + assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + + // Any ancestors before the shuffle are not considered + assert(ancestors4.size === 1) + assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) + assert(ancestors5.size === 4) + assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) + } + + test("getNarrowAncestors with multiple parents") { + val rdd1 = sc.parallelize(1 to 100, 5) + val rdd2 = sc.parallelize(1 to 200, 10).map(_ + 1) + val rdd3 = sc.parallelize(1 to 300, 15).filter(_ > 50) + val rdd4 = rdd1.map(i => (i, i)) + val rdd5 = rdd2.map(i => (i, i)) + val rdd6 = sc.union(rdd1, rdd2) + val rdd7 = sc.union(rdd1, rdd2, rdd3) + val rdd8 = sc.union(rdd6, rdd7) + val rdd9 = rdd4.join(rdd5) + val ancestors6 = rdd6.getNarrowAncestors + val ancestors7 = rdd7.getNarrowAncestors + val ancestors8 = rdd8.getNarrowAncestors + val ancestors9 = rdd9.getNarrowAncestors + + // Simple dependency tree with multiple branches + assert(ancestors6.size === 3) + assert(ancestors6.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 2) + assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors7.size === 5) + assert(ancestors7.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) + assert(ancestors7.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors7.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + + // Dependency tree with duplicate nodes (e.g. rdd1 should not be reported twice) + assert(ancestors8.size === 7) + assert(ancestors8.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors8.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors8.count(_.isInstanceOf[UnionRDD[_]]) === 2) + assert(ancestors8.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) + assert(ancestors8.count(_ == rdd1) === 1) + assert(ancestors8.count(_ == rdd2) === 1) + assert(ancestors8.count(_ == rdd3) === 1) + + // Any ancestors before the shuffle are not considered + assert(ancestors9.size === 2) + assert(ancestors9.count(_.isInstanceOf[CoGroupedRDD[_]]) === 1) + assert(ancestors9.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 1) + } + + /** + * This tests for the pathological condition in which the RDD dependency graph is cyclical. + * + * Since RDD is part of the public API, applications may actually implement RDDs that allow + * such graphs to be constructed. In such cases, getNarrowAncestor should not simply hang. + */ + test("getNarrowAncestors with cycles") { + val rdd1 = new CyclicalDependencyRDD[Int] + val rdd2 = new CyclicalDependencyRDD[Int] + val rdd3 = new CyclicalDependencyRDD[Int] + val rdd4 = rdd3.map(_ + 1).filter(_ > 10).map(_ + 2).filter(_ % 5 > 1) + val rdd5 = rdd4.map(_ + 2).filter(_ > 20) + val rdd6 = sc.union(rdd1, rdd2, rdd3).map(_ + 4).union(rdd5).union(rdd4) + + // Simple cyclical dependency + rdd1.addDependency(new OneToOneDependency[Int](rdd2)) + rdd2.addDependency(new OneToOneDependency[Int](rdd1)) + val ancestors1 = rdd1.getNarrowAncestors + val ancestors2 = rdd2.getNarrowAncestors + assert(ancestors1.size === 1) + assert(ancestors1.count(_ == rdd2) === 1) + assert(ancestors1.count(_ == rdd1) === 0) + assert(ancestors2.size === 1) + assert(ancestors2.count(_ == rdd1) === 1) + assert(ancestors2.count(_ == rdd2) === 0) + + // Cycle involving a longer chain + rdd3.addDependency(new OneToOneDependency[Int](rdd4)) + val ancestors3 = rdd3.getNarrowAncestors + val ancestors4 = rdd4.getNarrowAncestors + assert(ancestors3.size === 4) + assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_ == rdd3) === 0) + assert(ancestors4.size === 4) + assert(ancestors4.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors4.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors4.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) + assert(ancestors4.count(_ == rdd3) === 1) + assert(ancestors4.count(_ == rdd4) === 0) + + // Cycles that do not involve the root + val ancestors5 = rdd5.getNarrowAncestors + assert(ancestors5.size === 6) + assert(ancestors5.count(_.isInstanceOf[MappedRDD[_, _]]) === 3) + assert(ancestors5.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors5.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) + assert(ancestors4.count(_ == rdd3) === 1) + + // Complex cyclical dependency graph (combination of all of the above) + val ancestors6 = rdd6.getNarrowAncestors + assert(ancestors6.size === 12) + assert(ancestors6.count(_.isInstanceOf[UnionRDD[_]]) === 2) + assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 4) + assert(ancestors6.count(_.isInstanceOf[FilteredRDD[_]]) === 3) + assert(ancestors6.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 3) + } + + /** A contrived RDD that allows the manual addition of dependencies after creation. */ + private class CyclicalDependencyRDD[T: ClassTag] extends RDD[T](sc, Nil) { + private val mutableDependencies: ArrayBuffer[Dependency[_]] = ArrayBuffer.empty + override def compute(p: Partition, c: TaskContext): Iterator[T] = Iterator.empty + override def getPartitions: Array[Partition] = Array.empty + override def getDependencies: Seq[Dependency[_]] = mutableDependencies + def addDependency(dep: Dependency[_]) { + mutableDependencies += dep + } + } } 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 35a7ac9d049c2..d172dd1ac8e1b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -21,6 +21,8 @@ import scala.Tuple2 import scala.collection.mutable.{HashSet, HashMap, Map} import scala.language.reflectiveCalls +import akka.actor._ +import akka.testkit.{ImplicitSender, TestKit, TestActorRef} import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark._ @@ -28,19 +30,16 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -/** - * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler - * rather than spawning an event loop thread as happens in the real code. They use EasyMock - * to mock out two classes that DAGScheduler interacts with: TaskScheduler (to which TaskSets are - * submitted) and BlockManagerMaster (from which cache locations are retrieved and to which dead - * host notifications are sent). In addition, tests may check for side effects on a non-mocked - * MapOutputTracker instance. - * - * Tests primarily consist of running DAGScheduler#processEvent and - * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet) - * and capturing the resulting TaskSets from the mock TaskScheduler. - */ -class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { +class BuggyDAGEventProcessActor extends Actor { + val state = 0 + def receive = { + case _ => throw new SparkException("error") + } +} + +class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuite + with ImplicitSender with BeforeAndAfter with LocalSparkContext { + val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() @@ -58,7 +57,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) taskSets += taskSet } - override def cancelTasks(stageId: Int) { + override def cancelTasks(stageId: Int, interruptThread: Boolean) { cancelledStages += stageId } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} @@ -82,6 +81,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null + var dagEventProcessTestActor: TestActorRef[DAGSchedulerEventProcessActor] = null /** * Set of cache locations to return from our mock BlockManagerMaster. @@ -121,6 +121,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) scheduler = new DAGScheduler( + sc, taskScheduler, sc.listenerBus, mapOutputTracker, @@ -131,10 +132,13 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runLocallyWithinThread(job) } } + dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( + Props(classOf[DAGSchedulerEventProcessActor], scheduler))(system) } - after { - scheduler.stop() + override def afterAll() { + super.afterAll() + TestKit.shutdownActorSystem(system) } /** @@ -178,8 +182,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont * DAGScheduler event loop. */ private def runEvent(event: DAGSchedulerEvent) { - assert(!scheduler.processEvent(event)) - scheduler.submitWaitingStages() + dagEventProcessTestActor.receive(event) } /** @@ -209,7 +212,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont listener: JobListener = jobListener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) - return jobId + jobId } /** Sends TaskSetFailed to the scheduler. */ @@ -223,19 +226,17 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } test("zero split job") { - val rdd = makeRdd(0, Nil) var numResults = 0 val fakeListener = new JobListener() { override def taskSucceeded(partition: Int, value: Any) = numResults += 1 override def jobFailed(exception: Exception) = throw exception } - submit(rdd, Array(), listener = fakeListener) + submit(makeRdd(0, Nil), Array(), listener = fakeListener) assert(numResults === 0) } test("run trivial job") { - val rdd = makeRdd(1, Nil) - submit(rdd, Array(0)) + submit(makeRdd(1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty @@ -529,6 +530,18 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertDataStructuresEmpty } + test("DAGSchedulerActorSupervisor closes the SparkContext when EventProcessActor crashes") { + val actorSystem = ActorSystem("test") + val supervisor = actorSystem.actorOf( + Props(classOf[DAGSchedulerActorSupervisor], scheduler), "dagSupervisor") + supervisor ! Props[BuggyDAGEventProcessActor] + val child = expectMsgType[ActorRef] + watch(child) + child ! "hi" + expectMsgPF(){ case Terminated(child) => () } + assert(scheduler.sc.dagScheduler === null) + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. @@ -561,3 +574,4 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(scheduler.waitingStages.isEmpty) } } + 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 36511a9e95474..4e9fd07e68a21 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.SparkContext._ import org.apache.spark.executor.TaskMetrics class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers - with BeforeAndAfter with BeforeAndAfterAll { + with BeforeAndAfter with BeforeAndAfterAll { + /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 @@ -37,7 +38,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc = new SparkContext("local", "SparkListenerSuite") } - override def afterAll { + override def afterAll() { System.clearProperty("spark.akka.frameSize") } @@ -50,9 +51,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } assert(counter.count === 0) - // Starting listener bus should flush all buffered events (asynchronously, hence the sleep) + // Starting listener bus should flush all buffered events bus.start() - Thread.sleep(1000) + assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(counter.count === 5) // After listener bus has stopped, posting events should not increment counter @@ -133,20 +134,58 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val rdd1 = sc.parallelize(1 to 100, 4) val rdd2 = rdd1.map(_.toString) rdd2.setName("Target RDD") - rdd2.count + rdd2.count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} val (stageInfo, taskInfoMetrics) = listener.stageInfos.head - stageInfo.rddInfo.name should be {"Target RDD"} + stageInfo.rddInfos.size should be {2} + stageInfo.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo.rddInfos.exists(_.name == "Target RDD") should be {true} stageInfo.numTasks should be {4} - stageInfo.rddInfo.numPartitions should be {4} stageInfo.submissionTime should be ('defined) stageInfo.completionTime should be ('defined) taskInfoMetrics.length should be {4} } + test("basic creation of StageInfo with shuffle") { + val listener = new SaveStageAndTaskInfo + sc.addSparkListener(listener) + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.filter(_ % 2 == 0).map(i => (i, i)) + val rdd3 = rdd2.reduceByKey(_ + _) + rdd1.setName("Un") + rdd2.setName("Deux") + rdd3.setName("Trois") + + rdd1.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {1} + val stageInfo1 = listener.stageInfos.keys.find(_.stageId == 0).get + stageInfo1.rddInfos.size should be {1} // ParallelCollectionRDD + stageInfo1.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo1.rddInfos.exists(_.name == "Un") should be {true} + listener.stageInfos.clear() + + rdd2.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {1} + val stageInfo2 = listener.stageInfos.keys.find(_.stageId == 1).get + stageInfo2.rddInfos.size should be {3} // ParallelCollectionRDD, FilteredRDD, MappedRDD + stageInfo2.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo2.rddInfos.exists(_.name == "Deux") should be {true} + listener.stageInfos.clear() + + rdd3.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {2} // Shuffle map stage + result stage + val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 2).get + stageInfo3.rddInfos.size should be {2} // ShuffledRDD, MapPartitionsRDD + stageInfo3.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo3.rddInfos.exists(_.name == "Trois") should be {true} + } + test("StageInfo with fewer tasks than partitions") { val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) @@ -159,7 +198,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.size should be {1} val (stageInfo, _) = listener.stageInfos.head stageInfo.numTasks should be {2} - stageInfo.rddInfo.numPartitions should be {4} + stageInfo.rddInfos.size should be {2} + stageInfo.rddInfos.forall(_.numPartitions == 4) should be {true} } test("local metrics") { @@ -167,7 +207,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) // just to make sure some of the tasks take a noticeable amount of time - val w = {i:Int => + val w = { i: Int => if (i == 0) Thread.sleep(100) i @@ -199,7 +239,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc checkNonZeroAvg( taskInfoMetrics.map(_._2.executorDeserializeTime), stageInfo + " executorDeserializeTime") - if (stageInfo.rddInfo.name == d4.name) { + if (stageInfo.rddInfos.exists(_.name == d4.name)) { checkNonZeroAvg( taskInfoMetrics.map(_._2.shuffleReadMetrics.get.fetchWaitTime), stageInfo + " fetchWaitTime") @@ -207,11 +247,11 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) - if (stageInfo.rddInfo.name == d2.name || stageInfo.rddInfo.name == d3.name) { + if (stageInfo.rddInfos.exists(info => info.name == d2.name || info.name == d3.name)) { taskMetrics.shuffleWriteMetrics should be ('defined) taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) } - if (stageInfo.rddInfo.name == d4.name) { + if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get sm.totalBlocksFetched should be > (0) 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 2fb750d9ee378..a8b605c5b212a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -305,7 +305,7 @@ 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 907428db80af3..00deecc1c3ca9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.storage -import java.nio.ByteBuffer +import java.nio.{ByteBuffer, MappedByteBuffer} +import java.util.Arrays import akka.actor._ -import org.scalatest.BeforeAndAfter -import org.scalatest.FunSuite -import org.scalatest.PrivateMethodTester +import org.apache.spark.SparkConf +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import org.mockito.Mockito.{mock, when} +import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ @@ -785,6 +788,53 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } } + test("reads of memory-mapped and non memory-mapped files are equivalent") { + val confKey = "spark.storage.memoryMapThreshold" + + // Create a non-trivial (not all zeros) byte array + var counter = 0.toByte + def incr = {counter = (counter + 1).toByte; counter;} + val bytes = Array.fill[Byte](1000)(incr) + val byteBuffer = ByteBuffer.wrap(bytes) + + val blockId = BlockId("rdd_1_2") + + // This sequence of mocks makes these tests fairly brittle. It would + // be nice to refactor classes involved in disk storage in a way that + // allows for easier testing. + val blockManager = mock(classOf[BlockManager]) + val shuffleBlockManager = mock(classOf[ShuffleBlockManager]) + when(shuffleBlockManager.conf).thenReturn(conf) + val diskBlockManager = new DiskBlockManager(shuffleBlockManager, + System.getProperty("java.io.tmpdir")) + + when(blockManager.conf).thenReturn(conf.clone.set(confKey, 0.toString)) + val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) + diskStoreMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) + val mapped = diskStoreMapped.getBytes(blockId).get + + when(blockManager.conf).thenReturn(conf.clone.set(confKey, (1000 * 1000).toString)) + val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) + diskStoreNotMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) + val notMapped = diskStoreNotMapped.getBytes(blockId).get + + // Not possible to do isInstanceOf due to visibility of HeapByteBuffer + assert(notMapped.getClass.getName.endsWith("HeapByteBuffer"), + "Expected HeapByteBuffer for un-mapped read") + assert(mapped.isInstanceOf[MappedByteBuffer], "Expected MappedByteBuffer for mapped read") + + def arrayFromByteBuffer(in: ByteBuffer): Array[Byte] = { + val array = new Array[Byte](in.remaining()) + in.get(array) + array + } + + val mappedAsArray = arrayFromByteBuffer(mapped) + val notMappedAsArray = arrayFromByteBuffer(notMapped) + assert(Arrays.equals(mappedAsArray, bytes)) + assert(Arrays.equals(notMappedAsArray, bytes)) + } + test("updated block statuses") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, mapOutputTracker) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 9b29e2a8a55de..42bfbf1bdfc74 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -53,6 +53,11 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { shuffleBlockManager.idToSegmentMap.clear() } + override def afterEach() { + diskBlockManager.stop() + shuffleBlockManager.idToSegmentMap.clear() + } + test("basic block creation") { val blockId = new TestBlockId("test") assertSegmentEquals(blockId, blockId.name, 0, 0) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index ed02b0ba00d43..fff8020ade6d1 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -36,7 +36,7 @@ import scala.xml.Node class UISuite extends FunSuite { - test("basic ui visibility") { + ignore("basic ui visibility") { withSpark(new SparkContext("local", "test")) { sc => // test if the ui is visible, and all the expected tabs are visible eventually(timeout(10 seconds), interval(50 milliseconds)) { @@ -50,7 +50,7 @@ class UISuite extends FunSuite { } } - test("visibility at localhost:4040") { + ignore("visibility at localhost:4040") { withSpark(new SparkContext("local", "test")) { sc => // test if visible from http://localhost:4040 eventually(timeout(10 seconds), interval(50 milliseconds)) { @@ -60,7 +60,7 @@ class UISuite extends FunSuite { } } - test("attaching a new tab") { + ignore("attaching a new tab") { withSpark(new SparkContext("local", "test")) { sc => val sparkUI = sc.ui 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 16470bb7bf60d..3031015256ec9 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -231,7 +231,10 @@ class JsonProtocolSuite extends FunSuite { assert(info1.submissionTime === info2.submissionTime) assert(info1.completionTime === info2.completionTime) assert(info1.emittedTaskSizeWarning === info2.emittedTaskSizeWarning) - assertEquals(info1.rddInfo, info2.rddInfo) + assert(info1.rddInfos.size === info2.rddInfos.size) + (0 until info1.rddInfos.size).foreach { i => + assertEquals(info1.rddInfos(i), info2.rddInfos(i)) + } } private def assertEquals(info1: RDDInfo, info2: RDDInfo) { @@ -434,7 +437,8 @@ class JsonProtocolSuite extends FunSuite { } private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { - new StageInfo(a, "greetings", b, makeRddInfo(a, b, c, d, e)) + val rddInfos = (1 to a % 5).map { i => makeRddInfo(a % i, b % i, c % i, d % i, e % i) } + new StageInfo(a, "greetings", b, rddInfos) } private def makeTaskInfo(a: Long, b: Int, c: Long) = { diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index fa2f02dfecc75..4a816d4101e57 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -114,6 +114,7 @@ def ensure_path_not_present(x): os.environ["SPARK_VERSION"] = RELEASE_VERSION os.environ["SCALA_VERSION"] = SCALA_VERSION os.environ["SPARK_RELEASE_REPOSITORY"] = RELEASE_REPOSITORY +os.environ["SPARK_AUDIT_MASTER"] = "local" for module in modules: os.environ["SPARK_MODULE"] = module ret = run_cmd("sbt clean update", exit_on_failure=False) diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index 53fe43215e40e..a89b0d7d38bf1 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -24,8 +24,13 @@ import org.apache.spark.SparkContext._ object SimpleApp { def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Spark App").setMaster(master) + case None => new SparkConf().setAppName("Simple Spark App") + } val logFile = "input.txt" - val sc = new SparkContext("local", "Simple App") + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() diff --git a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala index da08e014ebd94..24c7f8d667296 100644 --- a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala +++ b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala @@ -17,14 +17,20 @@ package main.scala -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD object GraphXApp { def main(args: Array[String]) { - val sc = new SparkContext("local", "Simple GraphX App") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple GraphX App").setMaster(master) + case None => new SparkConf().setAppName("Simple Graphx App") + } + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) + val users: RDD[(VertexId, (String, String))] = sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")), diff --git a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala index 3d0722d2ac45e..a1d8971abe9a4 100644 --- a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala +++ b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala @@ -27,10 +27,12 @@ import org.apache.spark.streaming._ object SparkStreamingExample { def main(args: Array[String]) { - val conf = new SparkConf(true) - .setMaster("local[2]") - .setAppName("Streaming test") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Streaming App").setMaster(master) + case None => new SparkConf().setAppName("Simple Streaming App") + } val ssc = new StreamingContext(conf, Seconds(1)) + SparkContext.jarOfClass(this.getClass).foreach(ssc.sparkContext.addJar) val seen = ListBuffer[RDD[Int]]() val rdd1 = ssc.sparkContext.makeRDD(1 to 100, 10) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index bf1c5d7953bd2..33552a74920a7 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -21,47 +21,53 @@ # Publishes releases to Maven and packages/copies binary release artifacts. # Expects to be run in a totally empty directory. # +# Options: +# --package-only only packages an existing release candidate +# # Would be nice to add: # - Send output to stderr and have useful logging in stdout -# - Have this use sbt rather than Maven release plug in -GIT_USERNAME=pwendell -GIT_PASSWORD=XXX -GPG_PASSPHRASE=XXX -GIT_BRANCH=branch-0.9 -RELEASE_VERSION=0.9.0-incubating -RC_NAME=rc2 -USER_NAME=pwendell +GIT_USERNAME=${GIT_USERNAME:-pwendell} +GIT_PASSWORD=${GIT_PASSWORD:-XXX} +GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX} +GIT_BRANCH=${GIT_BRANCH:-branch-1.0} +RELEASE_VERSION=${RELEASE_VERSION:-1.0.0} +RC_NAME=${RC_NAME:-rc2} +USER_NAME=${USER_NAME:-pwendell} set -e -GIT_TAG=v$RELEASE_VERSION - -# Artifact publishing +GIT_TAG=v$RELEASE_VERSION-$RC_NAME -git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH -cd spark -export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" +if [[ ! "$@" =~ --package-only ]]; then + echo "Creating and publishing release" + # Artifact publishing + git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH + cd spark + export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" -mvn -Pyarn release:clean + mvn -Pyarn release:clean -mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ - -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ - --batch-mode release:prepare + mvn -DskipTests \ + -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ + -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Pyarn -Phive -Pspark-ganglia-lgpl\ + -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ + --batch-mode release:prepare -mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ - release:perform + mvn -DskipTests \ + -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Pyarn -Phive -Pspark-ganglia-lgpl\ + release:perform -rm -rf spark + cd .. + rm -rf spark +fi # Source and binary tarballs +echo "Packaging release tarballs" git clone https://git-wip-us.apache.org/repos/asf/spark.git cd spark git checkout --force $GIT_TAG @@ -83,16 +89,15 @@ rm -rf spark-$RELEASE_VERSION make_binary_release() { NAME=$1 - MAVEN_FLAGS=$2 - + FLAGS=$2 cp -r spark spark-$RELEASE_VERSION-bin-$NAME + cd spark-$RELEASE_VERSION-bin-$NAME - export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" - mvn $MAVEN_FLAGS -DskipTests clean package - find . -name test-classes -type d | xargs rm -rf - find . -name classes -type d | xargs rm -rf + ./make-distribution.sh $FLAGS --name $NAME --tgz cd .. - tar cvzf spark-$RELEASE_VERSION-bin-$NAME.tgz spark-$RELEASE_VERSION-bin-$NAME + cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . + rm -rf spark-$RELEASE_VERSION-bin-$NAME + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz @@ -102,19 +107,18 @@ make_binary_release() { echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ spark-$RELEASE_VERSION-bin-$NAME.tgz.sha - rm -rf spark-$RELEASE_VERSION-bin-$NAME } -make_binary_release "hadoop1" "-Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Dhadoop.version=2.0.0-mr1-cdh4.2.0" -make_binary_release "hadoop2" "-Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0" +make_binary_release "hadoop1" "--hadoop 1.0.4" +make_binary_release "cdh4" "--hadoop 2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop2" "--with-yarn --hadoop 2.2.0" # Copy data echo "Copying release tarballs" -ssh $USER_NAME@people.apache.org \ - mkdir /home/$USER_NAME/public_html/spark-$RELEASE_VERSION-$RC_NAME rc_folder=spark-$RELEASE_VERSION-$RC_NAME -scp spark* \ +ssh $USER_NAME@people.apache.org \ + mkdir /home/$USER_NAME/public_html/$rc_folder +scp spark-* \ $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/ # Docs @@ -123,7 +127,9 @@ cd docs PRODUCTION=1 jekyll build echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs -rsync -r _site/* $USER_NAME@people.apache.org /home/$USER_NAME/public_html/$rc_docs_folder +ssh $USER_NAME@people.apache.org \ + mkdir /home/$USER_NAME/public_html/$rc_docs_folder +rsync -r _site/* $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_docs_folder echo "Release $RELEASE_VERSION completed:" echo "Git tag:\t $GIT_TAG" diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 7a61943e94814..83618c8068d35 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -26,19 +26,33 @@ import json import os +import re import subprocess import sys import tempfile import urllib2 +try: + import jira.client + JIRA_IMPORTED=True +except ImportError: + JIRA_IMPORTED=False + # Location of your Spark git development area SPARK_HOME = os.environ.get("SPARK_HOME", "/home/patrick/Documents/spark") # Remote name which points to the Gihub site PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") - -GIT_API_BASE = "https://api.github.com/repos/apache/spark" +# ASF JIRA username +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell") +# ASF JIRA password +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "1234") + +GITHUB_BASE = "https://github.com/apache/spark/pull" +GITHUB_API_BASE = "https://api.github.com/repos/apache/spark" +JIRA_BASE = "https://issues.apache.org/jira/browse" +JIRA_API_BASE = "https://issues.apache.org/jira" # Prefix added to temporary branches BRANCH_PREFIX = "PR_TOOL" @@ -145,8 +159,7 @@ def merge_pr(pr_num, target_ref): return merge_hash -def maybe_cherry_pick(pr_num, merge_hash, default_branch): - continue_maybe("Would you like to pick %s into another branch?" % merge_hash) +def cherry_pick(pr_num, merge_hash, default_branch): pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) if pick_ref == "": pick_ref = default_branch @@ -171,14 +184,86 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) print("Pick hash: %s" % pick_hash) + return pick_ref + +def fix_version_from_branch(branch, versions): + # Note: Assumes this is a sorted (newest->oldest) list of un-released versions + if branch == "master": + return versions[0] + else: + branch_ver = branch.replace("branch-", "") + return filter(lambda x: x.name.startswith(branch_ver), versions)[-1] + +def resolve_jira(title, merge_branches, comment): + asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + + default_jira_id = "" + search = re.findall("SPARK-[0-9]{4,5}", title) + if len(search) > 0: + default_jira_id = search[0] + + jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) + if jira_id == "": + jira_id = default_jira_id -branches = get_json("%s/branches" % GIT_API_BASE) + try: + issue = asf_jira.issue(jira_id) + except Exception as e: + fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) + + cur_status = issue.fields.status.name + cur_summary = issue.fields.summary + cur_assignee = issue.fields.assignee + if cur_assignee == None: + cur_assignee = "NOT ASSIGNED!!!" + else: + cur_assignee = cur_assignee.displayName + + if cur_status == "Resolved" or cur_status == "Closed": + fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) + print ("=== JIRA %s ===" % jira_id) + print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( + cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) + + versions = asf_jira.project_versions("SPARK") + versions = sorted(versions, key = lambda x: x.name, reverse=True) + versions = filter(lambda x: x.raw['released'] == False, versions) + + default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) + for v in default_fix_versions: + # Handles the case where we have forked a release branch but not yet made the release. + # In this case, if the PR is committed to the master branch and the release branch, we + # only consider the release branch to be the fix version. E.g. it is not valid to have + # both 1.1.0 and 1.0.0 as fix versions. + (major, minor, patch) = v.split(".") + if patch == "0": + previous = "%s.%s.%s" % (major, int(minor) - 1, 0) + if previous in default_fix_versions: + default_fix_versions = filter(lambda x: x != v, default_fix_versions) + default_fix_versions = ",".join(default_fix_versions) + + fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) + if fix_versions == "": + fix_versions = default_fix_versions + fix_versions = fix_versions.replace(" ", "").split(",") + + def get_version_json(version_str): + return filter(lambda v: v.name == version_str, versions)[0].raw + jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) + + resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] + asf_jira.transition_issue(jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) + + print "Succesfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + +branches = get_json("%s/branches" % GITHUB_API_BASE) branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) # Assumes branch names can be sorted lexicographically latest_branch = sorted(branch_names, reverse=True)[0] pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") -pr = get_json("%s/pulls/%s" % (GIT_API_BASE, pr_num)) +pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) url = pr["url"] title = pr["title"] @@ -208,11 +293,22 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): continue_maybe(msg) print ("\n=== Pull Request #%s ===" % pr_num) -print("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( +print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( title, pr_repo_desc, target_ref, url)) continue_maybe("Proceed with merging pull request #%s?" % pr_num) +merged_refs = [target_ref] + merge_hash = merge_pr(pr_num, target_ref) -while True: - maybe_cherry_pick(pr_num, merge_hash, latest_branch) +pick_prompt = "Would you like to pick %s into another branch?" % merge_hash +while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] + +if JIRA_IMPORTED: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira(title, merged_refs, jira_comment) +else: + print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." + print "Exiting without trying to close the associated JIRA." diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 5d4dbb7a9c3a9..8b543de574622 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -76,32 +76,9 @@ @@ -140,33 +117,6 @@

{{ page.title }}

{{ content }} - - - - - @@ -174,42 +124,23 @@

Heading

- + + + - - - - - diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 05f0bd47a88a5..2dbbbf6feb4b8 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -20,47 +20,48 @@ if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1') # Build Scaladoc for Java/Scala - core_projects = ["core", "examples", "repl", "bagel", "graphx", "streaming", "mllib"] - external_projects = ["flume", "kafka", "mqtt", "twitter", "zeromq"] - sql_projects = ["catalyst", "core", "hive"] - projects = core_projects - projects = projects + external_projects.map { |project_name| "external/" + project_name } - projects = projects + sql_projects.map { |project_name| "sql/" + project_name } - - puts "Moving to project root and building scaladoc." + puts "Moving to project root and building API docs." curr_dir = pwd cd("..") - puts "Running 'sbt/sbt doc hive/doc' from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt doc hive/doc` + puts "Running 'sbt/sbt compile unidoc' from " + pwd + "; this may take a few minutes..." + puts `sbt/sbt compile unidoc` puts "Moving back into docs dir." cd("docs") - # Copy over the scaladoc from each project into the docs directory. + # Copy over the unified ScalaDoc for all projects to api/scala. # This directory will be copied over to _site when `jekyll` command is run. - projects.each do |project_name| - source = "../" + project_name + "/target/scala-2.10/api" - dest = "api/" + project_name + source = "../target/scala-2.10/unidoc" + dest = "api/scala" + + puts "Making directory " + dest + mkdir_p dest + + # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. + puts "cp -r " + source + "/. " + dest + cp_r(source + "/.", dest) + + # Append custom JavaScript + js = File.readlines("./js/api-docs.js") + js_file = dest + "/lib/template.js" + File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } - puts "making directory " + dest - mkdir_p dest + # Append custom CSS + css = File.readlines("./css/api-docs.css") + css_file = dest + "/lib/template.css" + File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } - # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. - puts "cp -r " + source + "/. " + dest - cp_r(source + "/.", dest) + # Copy over the unified JavaDoc for all projects to api/java. + source = "../target/javaunidoc" + dest = "api/java" - # Append custom JavaScript - js = File.readlines("./js/api-docs.js") - js_file = dest + "/lib/template.js" - File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } + puts "Making directory " + dest + mkdir_p dest - # Append custom CSS - css = File.readlines("./css/api-docs.css") - css_file = dest + "/lib/template.css" - File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } - end + puts "cp -r " + source + "/. " + dest + cp_r(source + "/.", dest) # Build Epydoc for Python puts "Moving to python directory and building epydoc." @@ -70,11 +71,11 @@ puts "Moving back into docs dir." cd("../docs") - puts "echo making directory pyspark" - mkdir_p "pyspark" + puts "Making directory api/python" + mkdir_p "api/python" - puts "cp -r ../python/docs/. api/pyspark" - cp_r("../python/docs/.", "api/pyspark") + puts "cp -r ../python/docs/. api/python" + cp_r("../python/docs/.", "api/python") cd("..") end diff --git a/docs/api.md b/docs/api.md index 91c8e51d26c15..03460383335e8 100644 --- a/docs/api.md +++ b/docs/api.md @@ -1,13 +1,10 @@ --- layout: global -title: Spark API documentation (Scaladoc) +title: Spark API Documentation --- -Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory. +Here you can API docs for Spark and its submodules. -- [Spark](api/core/index.html) -- [Spark Examples](api/examples/index.html) -- [Spark Streaming](api/streaming/index.html) -- [Bagel](api/bagel/index.html) -- [GraphX](api/graphx/index.html) -- [PySpark](api/pyspark/index.html) +- [Spark Scala API (Scaladoc)](api/scala/index.html) +- [Spark Java API (Javadoc)](api/java/index.html) +- [Spark Python API (Epydoc)](api/python/index.html) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 9cebaf12283fc..a5e5303467401 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -39,17 +39,23 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit # Cloudera CDH 4.2.0 with MapReduce v1 $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property: + # Apache Hadoop 0.23.x + $ mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package + +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property. Note that Hadoop 0.23.X requires a special `-Phadoop-0.23` profile: # Apache Hadoop 2.0.5-alpha $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package # Cloudera CDH 4.2.0 with MapReduce v2 - $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package + $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-cdh4.2.0 -DskipTests clean package - # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer + # Apache Hadoop 2.2.X (e.g. 2.2.0 as below) and newer $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package + # Apache Hadoop 0.23.x + $ mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -Dyarn.version=0.23.7 -DskipTests clean package + ## 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). Some of the require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. You can then run the tests with `mvn -Dhadoop.version=... test`. diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 7f75ea44e4cea..b011679fede2d 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -50,61 +50,82 @@ The system currently supports three cluster managers: In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone cluster on Amazon EC2. -# Launching Applications - -The recommended way to launch a compiled Spark application is through the spark-submit script (located in the -bin directory), which takes care of setting up the classpath with Spark and its dependencies, as well as -provides a layer over the different cluster managers and deploy modes that Spark supports. It's usage is - - spark-submit `` `` - -Where options are any of: - -- **\--class** - The main class to run. -- **\--master** - The URL of the cluster manager master, e.g. spark://host:port, mesos://host:port, yarn, - or local. -- **\--deploy-mode** - "client" to run the driver in the client process or "cluster" to run the driver in - a process on the cluster. For Mesos, only "client" is supported. -- **\--executor-memory** - Memory per executor (e.g. 1000M, 2G). -- **\--executor-cores** - Number of cores per executor. (Default: 2) -- **\--driver-memory** - Memory for driver (e.g. 1000M, 2G) -- **\--name** - Name of the application. -- **\--arg** - Argument to be passed to the application's main class. This option can be specified - multiple times to pass multiple arguments. -- **\--jars** - A comma-separated list of local jars to include on the driver classpath and that - SparkContext.addJar will work with. Doesn't work on standalone with 'cluster' deploy mode. - -The following currently only work for Spark standalone with cluster deploy mode: - -- **\--driver-cores** - Cores for driver (Default: 1). -- **\--supervise** - If given, restarts the driver on failure. - -The following only works for Spark standalone and Mesos only: - -- **\--total-executor-cores** - Total cores for all executors. - -The following currently only work for YARN: - -- **\--queue** - The YARN queue to place the application in. -- **\--files** - Comma separated list of files to be placed in the working dir of each executor. -- **\--archives** - Comma separated list of archives to be extracted into the working dir of each - executor. -- **\--num-executors** - Number of executors (Default: 2). - -The master and deploy mode can also be set with the MASTER and DEPLOY_MODE environment variables. -Values for these options passed via command line will override the environment variables. - -# Shipping Code to the Cluster - -The recommended way to ship your code to the cluster is to pass it through SparkContext's constructor, -which takes a list of JAR files (Java/Scala) or .egg and .zip libraries (Python) to disseminate to -worker nodes. You can also dynamically add new files to be sent to executors with `SparkContext.addJar` -and `addFile`. - -## URIs for addJar / addFile - -- **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and every executor - pulls the file from the driver HTTP server +# Bundling and Launching Applications + +### 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 +[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 +as `provided` dependencies; these need not be bundled since they are provided by +the cluster manager at runtime. Once you have an assembled jar you can call the `bin/spark-submit` +script as shown here while passing your jar. + +For Python, you can use the `pyFiles` argument of SparkContext +or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. + +### Launching Applications with ./bin/spark-submit + +Once a user application is bundled, it can be launched using the `spark-submit` script located in +the bin directory. This script takes care of setting up the classpath with Spark and its +dependencies, and can support different cluster managers and deploy modes that Spark supports. +It's usage is + + ./bin/spark-submit --class path.to.your.Class [options] [app options] + +When calling `spark-submit`, `[app options]` will be passed along to your application's +main class. To enumerate all options available to `spark-submit` run it with +the `--help` flag. Here are a few examples of common options: + +{% highlight bash %} +# Run application locally +./bin/spark-submit \ + --class my.main.ClassName + --master local[8] \ + my-app.jar + +# Run on a Spark cluster +./bin/spark-submit \ + --class my.main.ClassName + --master spark://mycluster:7077 \ + --executor-memory 20G \ + --total-executor-cores 100 \ + my-app.jar + +# Run on a YARN cluster +HADOOP_CONF_DIR=XX /bin/spark-submit \ + --class my.main.ClassName + --master yarn-cluster \ # can also be `yarn-client` for client mode + --executor-memory 20G \ + --num-executors 50 \ + my-app.jar +{% endhighlight %} + +### Loading Configurations from a File + +The `spark-submit` script can load default `SparkConf` values from a properties file and pass them +onto your application. By default it will read configuration options from +`conf/spark-defaults.conf`. Any values specified in the file will be passed on to the +application when run. They can obviate the need for certain flags to `spark-submit`: for +instance, if `spark.master` property is set, you can safely omit the +`--master` flag from `spark-submit`. In general, configuration values explicitly set on a +`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values +in the defaults file. + +If you are ever unclear where configuration options are coming from. fine-grained debugging +information can be printed by adding the `--verbose` option to `./spark-submit`. + +### Advanced Dependency Management +When using `./bin/spark-submit` jars will be automatically transferred to the cluster. For many +users this is sufficient. However, advanced users can add jars by calling `addFile` or `addJar` +on an existing SparkContext. This can be used to distribute JAR files (Java/Scala) or .egg and +.zip libraries (Python) to executors. Spark uses the following URL scheme to allow different +strategies for disseminating jars: + +- **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and + every executor pulls the file from the driver HTTP server - **hdfs:**, **http:**, **https:**, **ftp:** - these pull down files and JARs from the URI as expected - **local:** - a URI starting with local:/ is expected to exist as a local file on each worker node. This means that no network IO will be incurred, and works well for large files/JARs that are pushed to each worker, @@ -138,6 +159,14 @@ The following table summarizes terms you'll see used to refer to cluster concept
+ + + + diff --git a/docs/configuration.md b/docs/configuration.md index a3029837ff0cd..b078c7c1112c2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -6,7 +6,7 @@ title: Spark Configuration Spark provides three locations to configure the system: * [Spark properties](#spark-properties) control most application parameters and can be set by passing - a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java + a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java system properties. * [Environment variables](#environment-variables) can be used to set per-machine settings, such as the IP address, through the `conf/spark-env.sh` script on each node. @@ -16,7 +16,7 @@ Spark provides three locations to configure the system: # Spark Properties Spark properties control most application settings and are configured separately for each application. -The preferred way to set them is by passing a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) +The preferred way to set them is by passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) class to your SparkContext constructor. Alternatively, Spark will also load them from Java system properties, for compatibility with old versions of Spark. @@ -53,7 +53,7 @@ there are at least five properties that you will commonly want to control: in serialized form. The default of Java serialization works with any Serializable Java object but is quite slow, so we recommend using org.apache.spark.serializer.KryoSerializer and configuring Kryo serialization when speed is necessary. Can be any subclass of - org.apache.spark.Serializer. + org.apache.spark.Serializer. @@ -62,7 +62,7 @@ there are at least five properties that you will commonly want to control: @@ -73,6 +73,9 @@ there are at least five properties that you will commonly want to control: Directory to use for "scratch" space in Spark, including map output files and RDDs that get stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks. + + NOTE: In Spark 1.0 and later this will be overriden by SPARK_LOCAL_DIRS (Standalone, Mesos) or + LOCAL_DIRS (YARN) envrionment variables set by the cluster manager. @@ -128,6 +131,15 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryFraction. + + + + + @@ -466,10 +478,13 @@ Apart from these, the following properties are also available, and may be useful - + @@ -578,7 +593,7 @@ Apart from these, the following properties are also available, and may be useful to consolidate them onto as few nodes as possible. Spreading out is usually better for data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. @@ -591,7 +606,7 @@ Apart from these, the following properties are also available, and may be useful Set this lower on a shared cluster to prevent users from grabbing the whole cluster by default.
Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. @@ -649,6 +664,34 @@ Apart from these, the following properties are also available, and may be useful Number of cores to allocate for each task. + + + + + + + + + + + + + + + +
Stage IdPool NameDescriptionSubmittedDurationTasks: Succeeded/TotalShuffle ReadShuffle Write
{s.stageId} - - {poolName.get} - - {makeDescription(s)}{submissionTime}{formattedDuration} - {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} + {s.stageId} + + {poolName.get} + {shuffleRead}{shuffleWrite}
{makeDescription(s)}{submissionTime}{formattedDuration} + {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} + {shuffleRead}{shuffleWrite}
Failure Reason{s.failureReason.getOrElse("")}Application User program built on Spark. Consists of a driver program and executors on the cluster.
Application jar + A jar containing the user's Spark application. In some cases users will want to create + an "uber jar" containing their application along with its dependencies. The user's jar + should never include Hadoop or Spark libraries, however, these will be added at runtime. +
Driver program The process running the main() function of the application and creating the SparkContext
If you use Kryo serialization, set this class to register your custom classes with Kryo. It should be set to a class that extends - KryoRegistrator. + KryoRegistrator. See the tuning guide for more details.
spark.storage.memoryMapThreshold8192 + Size of a block, in bytes, above which Spark memory maps when reading a block from disk. + This prevents Spark from memory mapping very small blocks. In general, memory + mapping has high overhead for blocks close to or below the page size of the operating system. +
spark.tachyonStore.baseDir System.getProperty("java.io.tmpdir")
spark.streaming.unpersistfalsetrue Force RDDs generated and persisted by Spark Streaming to be automatically unpersisted from - Spark's memory. Setting this to true is likely to reduce Spark's RDD memory usage. + Spark's memory. The raw input data received by Spark Streaming is also automatically cleared. + Setting this to false will allow the raw data and persisted RDDs to be accessible outside the + streaming application as they will not be cleared automatically. But it comes at the cost of + higher memory usage in Spark.
spark.executor.extraJavaOptions(none) + A string of extra JVM options to pass to executors. For instance, GC settings or other + logging. Note that it is illegal to set Spark properties or heap size settings with this + option. Spark properties should be set using a SparkConf object or the + spark-defaults.conf file used with the spark-submit script. Heap size settings can be set + with spark.executor.memory. +
spark.executor.extraClassPath(none) + Extra classpath entries to append to the classpath of executors. This exists primarily + for backwards-compatibility with older versions of Spark. Users typically should not need + to set this option. +
spark.executor.extraLibraryPath(none) + Set a special library path to use when launching executor JVM's. +
## Viewing Spark Properties @@ -659,10 +702,9 @@ This is a useful place to check to make sure that your properties have been set # Environment Variables Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh` -script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). These variables are meant to be for machine-specific settings, such -as library search paths. While Spark properties can also be set there through `SPARK_JAVA_OPTS`, for per-application settings, we recommend setting -these properties within the application instead of in `spark-env.sh` so that different applications can use different -settings. +script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). In Standalone and Mesos modes, +this file can give machine specific information such as hostnames. It is also sourced when running local +Spark applications or submission scripts. Note that `conf/spark-env.sh` does not exist by default when Spark is installed. However, you can copy `conf/spark-env.sh.template` to create it. Make sure you make the copy executable. @@ -672,13 +714,7 @@ The following variables can be set in `spark-env.sh`: * `JAVA_HOME`, the location where Java is installed (if it's not on your default `PATH`) * `PYSPARK_PYTHON`, the Python binary to use for PySpark * `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. -* `SPARK_LIBRARY_PATH`, to add search directories for native libraries. -* `SPARK_CLASSPATH`, to add elements to Spark's classpath that you want to be present for _all_ applications. - Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend - doing that when possible. -* `SPARK_JAVA_OPTS`, to add JVM options. This includes Java options like garbage collector settings and any system - properties that you'd like to pass with `-D`. One use case is to set some Spark properties differently on this - machine, e.g., `-Dspark.local.dir=/disk1,/disk2`. +* `SPARK_PUBLIC_DNS`, the hostname your Spark program will advertise to other machines. * Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each machine and maximum memory. diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 1238e3e0a4e7d..07be8ba58efa3 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -17,7 +17,7 @@ title: GraphX Programming Guide # Overview GraphX is the new (alpha) Spark API for graphs and graph-parallel computation. At a high-level, -GraphX extends the Spark [RDD](api/core/index.html#org.apache.spark.rdd.RDD) by introducing the +GraphX extends the Spark [RDD](api/scala/index.html#org.apache.spark.rdd.RDD) by introducing the [Resilient Distributed Property Graph](#property_graph): a directed multigraph with properties attached to each vertex and edge. To support graph computation, GraphX exposes a set of fundamental operators (e.g., [subgraph](#structural_operators), [joinVertices](#join_operators), and @@ -82,7 +82,7 @@ Prior to the release of GraphX, graph computation in Spark was expressed using B implementation of Pregel. GraphX improves upon Bagel by exposing a richer property graph API, a more streamlined version of the Pregel abstraction, and system optimizations to improve performance and reduce memory overhead. While we plan to eventually deprecate Bagel, we will continue to -support the [Bagel API](api/bagel/index.html#org.apache.spark.bagel.package) and +support the [Bagel API](api/scala/index.html#org.apache.spark.bagel.package) and [Bagel programming guide](bagel-programming-guide.html). However, we encourage Bagel users to explore the new GraphX API and comment on issues that may complicate the transition from Bagel. @@ -103,7 +103,7 @@ getting started with Spark refer to the [Spark Quick Start Guide](quick-start.ht # The Property Graph -The [property graph](api/graphx/index.html#org.apache.spark.graphx.Graph) is a directed multigraph +The [property graph](api/scala/index.html#org.apache.spark.graphx.Graph) is a directed multigraph with user defined objects attached to each vertex and edge. A directed multigraph is a directed graph with potentially multiple parallel edges sharing the same source and destination vertex. The ability to support parallel edges simplifies modeling scenarios where there can be multiple @@ -179,7 +179,7 @@ val userGraph: Graph[(String, String), String] There are numerous ways to construct a property graph from raw files, RDDs, and even synthetic generators and these are discussed in more detail in the section on [graph builders](#graph_builders). Probably the most general method is to use the -[Graph object](api/graphx/index.html#org.apache.spark.graphx.Graph$). For example the following +[Graph object](api/scala/index.html#org.apache.spark.graphx.Graph$). For example the following code constructs a graph from a collection of RDDs: {% highlight scala %} @@ -203,7 +203,7 @@ In the above example we make use of the [`Edge`][Edge] case class. Edges have a `dstId` corresponding to the source and destination vertex identifiers. In addition, the `Edge` class has an `attr` member which stores the edge property. -[Edge]: api/graphx/index.html#org.apache.spark.graphx.Edge +[Edge]: api/scala/index.html#org.apache.spark.graphx.Edge We can deconstruct a graph into the respective vertex and edge views by using the `graph.vertices` and `graph.edges` members respectively. @@ -229,7 +229,7 @@ The triplet view logically joins the vertex and edge properties yielding an `RDD[EdgeTriplet[VD, ED]]` containing instances of the [`EdgeTriplet`][EdgeTriplet] class. This *join* can be expressed in the following SQL expression: -[EdgeTriplet]: api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet +[EdgeTriplet]: api/scala/index.html#org.apache.spark.graphx.EdgeTriplet {% highlight sql %} SELECT src.id, dst.id, src.attr, e.attr, dst.attr @@ -270,8 +270,8 @@ core operators are defined in [`GraphOps`][GraphOps]. However, thanks to Scala operators in `GraphOps` are automatically available as members of `Graph`. For example, we can compute the in-degree of each vertex (defined in `GraphOps`) by the following: -[Graph]: api/graphx/index.html#org.apache.spark.graphx.Graph -[GraphOps]: api/graphx/index.html#org.apache.spark.graphx.GraphOps +[Graph]: api/scala/index.html#org.apache.spark.graphx.Graph +[GraphOps]: api/scala/index.html#org.apache.spark.graphx.GraphOps {% highlight scala %} val graph: Graph[(String, String), String] @@ -382,7 +382,7 @@ val newGraph = Graph(newVertices, graph.edges) val newGraph = graph.mapVertices((id, attr) => mapUdf(id, attr)) {% endhighlight %} -[Graph.mapVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexId,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED] +[Graph.mapVertices]: api/scala/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexId,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED] These operators are often used to initialize the graph for a particular computation or project away unnecessary properties. For example, given a graph with the out-degrees as the vertex properties @@ -419,7 +419,7 @@ This can be useful when, for example, trying to compute the inverse PageRank. B operation does not modify vertex or edge properties or change the number of edges, it can be implemented efficiently without data-movement or duplication. -[Graph.reverse]: api/graphx/index.html#org.apache.spark.graphx.Graph@reverse:Graph[VD,ED] +[Graph.reverse]: api/scala/index.html#org.apache.spark.graphx.Graph@reverse:Graph[VD,ED] The [`subgraph`][Graph.subgraph] operator takes vertex and edge predicates and returns the graph containing only the vertices that satisfy the vertex predicate (evaluate to true) and edges that @@ -427,7 +427,7 @@ satisfy the edge predicate *and connect vertices that satisfy the vertex predica operator can be used in number of situations to restrict the graph to the vertices and edges of interest or eliminate broken links. For example in the following code we remove broken links: -[Graph.subgraph]: api/graphx/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexId,VD)⇒Boolean):Graph[VD,ED] +[Graph.subgraph]: api/scala/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexId,VD)⇒Boolean):Graph[VD,ED] {% highlight scala %} // Create an RDD for the vertices @@ -467,7 +467,7 @@ vertices and edges that are also found in the input graph. This can be used in example, we might run connected components using the graph with missing vertices and then restrict the answer to the valid subgraph. -[Graph.mask]: api/graphx/index.html#org.apache.spark.graphx.Graph@mask[VD2,ED2](Graph[VD2,ED2])(ClassTag[VD2],ClassTag[ED2]):Graph[VD,ED] +[Graph.mask]: api/scala/index.html#org.apache.spark.graphx.Graph@mask[VD2,ED2](Graph[VD2,ED2])(ClassTag[VD2],ClassTag[ED2]):Graph[VD,ED] {% highlight scala %} // Run Connected Components @@ -482,7 +482,7 @@ The [`groupEdges`][Graph.groupEdges] operator merges parallel edges (i.e., dupli pairs of vertices) in the multigraph. In many numerical applications, parallel edges can be *added* (their weights combined) into a single edge thereby reducing the size of the graph. -[Graph.groupEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph@groupEdges((ED,ED)⇒ED):Graph[VD,ED] +[Graph.groupEdges]: api/scala/index.html#org.apache.spark.graphx.Graph@groupEdges((ED,ED)⇒ED):Graph[VD,ED] ## Join Operators @@ -506,7 +506,7 @@ returns a new graph with the vertex properties obtained by applying the user def to the result of the joined vertices. Vertices without a matching value in the RDD retain their original value. -[GraphOps.joinVertices]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexId,U)])((VertexId,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED] +[GraphOps.joinVertices]: api/scala/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexId,U)])((VertexId,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED] > Note that if the RDD contains more than one value for a given vertex only one will be used. It > is therefore recommended that the input RDD be first made unique using the following which will @@ -525,7 +525,7 @@ property type. Because not all vertices may have a matching value in the input function takes an `Option` type. For example, we can setup a graph for PageRank by initializing vertex properties with their `outDegree`. -[Graph.outerJoinVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexId,U)])((VertexId,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED] +[Graph.outerJoinVertices]: api/scala/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexId,U)])((VertexId,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED] {% highlight scala %} @@ -559,7 +559,7 @@ PageRank Value, shortest path to the source, and smallest reachable vertex id). ### Map Reduce Triplets (mapReduceTriplets) -[Graph.mapReduceTriplets]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=>Iterator[(org.apache.spark.graphx.VertexId,A)],reduceFunc:(A,A)=>A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A] +[Graph.mapReduceTriplets]: api/scala/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=>Iterator[(org.apache.spark.graphx.VertexId,A)],reduceFunc:(A,A)=>A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A] The core (heavily optimized) aggregation primitive in GraphX is the [`mapReduceTriplets`][Graph.mapReduceTriplets] operator: @@ -665,8 +665,8 @@ attributes at each vertex. This can be easily accomplished using the [`collectNeighborIds`][GraphOps.collectNeighborIds] and the [`collectNeighbors`][GraphOps.collectNeighbors] operators. -[GraphOps.collectNeighborIds]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexId]] -[GraphOps.collectNeighbors]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexId,VD)]] +[GraphOps.collectNeighborIds]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexId]] +[GraphOps.collectNeighbors]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexId,VD)]] {% highlight scala %} @@ -685,7 +685,7 @@ class GraphOps[VD, ED] { In Spark, RDDs are not persisted in memory by default. To avoid recomputation, they must be explicitly cached when using them multiple times (see the [Spark Programming Guide][RDD Persistence]). Graphs in GraphX behave the same way. **When using a graph multiple times, make sure to call [`Graph.cache()`][Graph.cache] on it first.** [RDD Persistence]: scala-programming-guide.html#rdd-persistence -[Graph.cache]: api/graphx/index.html#org.apache.spark.graphx.Graph@cache():Graph[VD,ED] +[Graph.cache]: api/scala/index.html#org.apache.spark.graphx.Graph@cache():Graph[VD,ED] In iterative computations, *uncaching* may also be necessary for best performance. By default, cached RDDs and graphs will remain in memory until memory pressure forces them to be evicted in LRU order. For iterative computation, intermediate results from previous iterations will fill up the cache. Though they will eventually be evicted, the unnecessary data stored in memory will slow down garbage collection. It would be more efficient to uncache intermediate results as soon as they are no longer necessary. This involves materializing (caching and forcing) a graph or RDD every iteration, uncaching all other datasets, and only using the materialized dataset in future iterations. However, because graphs are composed of multiple RDDs, it can be difficult to unpersist them correctly. **For iterative computation we recommend using the Pregel API, which correctly unpersists intermediate results.** @@ -716,7 +716,7 @@ messages remaining. The following is the type signature of the [Pregel operator][GraphOps.pregel] as well as a *sketch* of its implementation (note calls to graph.cache have been removed): -[GraphOps.pregel]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexId,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexId,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED] +[GraphOps.pregel]: api/scala/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexId,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexId,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED] {% highlight scala %} class GraphOps[VD, ED] { @@ -840,12 +840,12 @@ object Graph { [`Graph.fromEdgeTuples`][Graph.fromEdgeTuples] allows creating a graph from only an RDD of edge tuples, assigning the edges the value 1, and automatically creating any vertices mentioned by edges and assigning them the default value. It also supports deduplicating the edges; to deduplicate, pass `Some` of a [`PartitionStrategy`][PartitionStrategy] as the `uniqueEdges` parameter (for example, `uniqueEdges = Some(PartitionStrategy.RandomVertexCut)`). A partition strategy is necessary to colocate identical edges on the same partition so they can be deduplicated. -[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy$ +[PartitionStrategy]: api/scala/index.html#org.apache.spark.graphx.PartitionStrategy$ -[GraphLoader.edgeListFile]: api/graphx/index.html#org.apache.spark.graphx.GraphLoader$@edgeListFile(SparkContext,String,Boolean,Int):Graph[Int,Int] -[Graph.apply]: api/graphx/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexId,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] -[Graph.fromEdgeTuples]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexId,VertexId)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int] -[Graph.fromEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdges[VD,ED](RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] +[GraphLoader.edgeListFile]: api/scala/index.html#org.apache.spark.graphx.GraphLoader$@edgeListFile(SparkContext,String,Boolean,Int):Graph[Int,Int] +[Graph.apply]: api/scala/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexId,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] +[Graph.fromEdgeTuples]: api/scala/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexId,VertexId)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int] +[Graph.fromEdges]: api/scala/index.html#org.apache.spark.graphx.Graph$@fromEdges[VD,ED](RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] # Vertex and Edge RDDs @@ -913,7 +913,7 @@ of the various partitioning strategies defined in [`PartitionStrategy`][Partitio each partition, edge attributes and adjacency structure, are stored separately enabling maximum reuse when changing attribute values. -[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy +[PartitionStrategy]: api/scala/index.html#org.apache.spark.graphx.PartitionStrategy The three additional functions exposed by the `EdgeRDD` are: {% highlight scala %} @@ -952,7 +952,7 @@ the [`Graph.partitionBy`][Graph.partitionBy] operator. The default partitioning the initial partitioning of the edges as provided on graph construction. However, users can easily switch to 2D-partitioning or other heuristics included in GraphX. -[Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph$@partitionBy(partitionStrategy:org.apache.spark.graphx.PartitionStrategy):org.apache.spark.graphx.Graph[VD,ED] +[Graph.partitionBy]: api/scala/index.html#org.apache.spark.graphx.Graph$@partitionBy(partitionStrategy:org.apache.spark.graphx.PartitionStrategy):org.apache.spark.graphx.Graph[VD,ED]

ClassFunction Type @@ -81,7 +81,7 @@ interface has a single abstract method, `call()`, that must be implemented. ## Storage Levels RDD [storage level](scala-programming-guide.html#rdd-persistence) constants, such as `MEMORY_AND_DISK`, are -declared in the [org.apache.spark.api.java.StorageLevels](api/core/index.html#org.apache.spark.api.java.StorageLevels) class. To +declared in the [org.apache.spark.api.java.StorageLevels](api/java/index.html?org/apache/spark/api/java/StorageLevels.html) class. To define your own storage level, you can use StorageLevels.create(...). # Other Features @@ -101,11 +101,11 @@ the following changes: classes to interfaces. This means that concrete implementations of these `Function` classes will need to use `implements` rather than `extends`. * Certain transformation functions now have multiple versions depending - on the return type. In Spark core, the map functions (map, flatMap, - mapPartitons) have type-specific versions, e.g. - [`mapToPair`](api/core/index.html#org.apache.spark.api.java.JavaRDD@mapToPair[K2,V2](f:org.apache.spark.api.java.function.PairFunction[T,K2,V2]):org.apache.spark.api.java.JavaPairRDD[K2,V2]) - and [`mapToDouble`](api/core/index.html#org.apache.spark.api.java.JavaRDD@mapToDouble[R](f:org.apache.spark.api.java.function.DoubleFunction[T]):org.apache.spark.api.java.JavaDoubleRDD). - Spark Streaming also uses the same approach, e.g. [`transformToPair`](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaDStream@transformToPair[K2,V2](transformFunc:org.apache.spark.api.java.function.Function[R,org.apache.spark.api.java.JavaPairRDD[K2,V2]]):org.apache.spark.streaming.api.java.JavaPairDStream[K2,V2]). + on the return type. In Spark core, the map functions (`map`, `flatMap`, and + `mapPartitons`) have type-specific versions, e.g. + [`mapToPair`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToPair(org.apache.spark.api.java.function.PairFunction)) + and [`mapToDouble`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToDouble(org.apache.spark.api.java.function.DoubleFunction)). + Spark Streaming also uses the same approach, e.g. [`transformToPair`](api/java/org/apache/spark/streaming/api/java/JavaDStreamLike.html#transformToPair(org.apache.spark.api.java.function.Function)). # Example @@ -205,16 +205,9 @@ JavaPairRDD counts = lines.flatMapToPair( There is no performance difference between these approaches; the choice is just a matter of style. -# Javadoc - -We currently provide documentation for the Java API as Scaladoc, in the -[`org.apache.spark.api.java` package](api/core/index.html#org.apache.spark.api.java.package), because -some of the classes are implemented in Scala. It is important to note that the types and function -definitions show Scala syntax (for example, `def reduce(func: Function2[T, T]): T` instead of -`T reduce(Function2 func)`). In addition, the Scala `trait` modifier is used for Java -interface classes. We hope to generate documentation with Java-style syntax in the future to -avoid these quirks. +# API Docs +[API documentation](api/java/index.html) for Spark in Java is available in Javadoc format. # Where to Go from Here diff --git a/docs/js/main.js b/docs/js/main.js index 0bd2286cced19..59055467110bf 100755 --- a/docs/js/main.js +++ b/docs/js/main.js @@ -73,8 +73,26 @@ function viewSolution() { }); } +// A script to fix internal hash links because we have an overlapping top bar. +// Based on https://github.com/twitter/bootstrap/issues/193#issuecomment-2281510 +function maybeScrollToHash() { + console.log("HERE"); + if (window.location.hash && $(window.location.hash).length) { + console.log("HERE2", $(window.location.hash), $(window.location.hash).offset().top); + var newTop = $(window.location.hash).offset().top - 57; + $(window).scrollTop(newTop); + } +} $(function() { codeTabs(); viewSolution(); + + $(window).bind('hashchange', function() { + maybeScrollToHash(); + }); + + // Scroll now too in case we had opened the page on a hash, but wait a bit because some browsers + // will try to do *their* initial scroll after running the onReady handler. + $(window).load(function() { setTimeout(function() { maybeScrollToHash(); }, 25); }); }); diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md new file mode 100644 index 0000000000000..710ce1721fe25 --- /dev/null +++ b/docs/mllib-basics.md @@ -0,0 +1,476 @@ +--- +layout: global +title: MLlib - Basics +--- + +* Table of contents +{:toc} + +MLlib supports local vectors and matrices stored on a single machine, +as well as distributed matrices backed by one or more RDDs. +In the current implementation, local vectors and matrices are simple data models +to serve public interfaces. The underly linear algebra operations are provided by +[Breeze](http://www.scalanlp.org/) and [jblas](http://jblas.org/). +A training example used in supervised learning is called "labeled point" in MLlib. + +## Local vector + +A local vector has integer-typed and 0-based indices and double-typed values, stored on a single +machine. MLlib supports two types of local vectors: dense and sparse. A dense vector is backed by +a double array representing its entry values, while a sparse vector is backed by two parallel +arrays: indices and values. For example, a vector $(1.0, 0.0, 3.0)$ can be represented in dense +format as `[1.0, 0.0, 3.0]` or in sparse format as `(3, [0, 2], [1.0, 3.0])`, where `3` is the size +of the vector. + +

+
+ +The base class of local vectors is +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector), and we provide two +implementations: [`DenseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseVector) and +[`SparseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.SparseVector). We recommend +using the factory methods implemented in +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) to create local vectors. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +// Create a dense vector (1.0, 0.0, 3.0). +val dv: Vector = Vectors.dense(1.0, 0.0, 3.0) +// Create a sparse vector (1.0, 0.0, 3.0) by specifying its indices and values corresponding to nonzero entries. +val sv1: Vector = Vectors.sparse(3, Array(0, 2), Array(1.0, 3.0)) +// Create a sparse vector (1.0, 0.0, 3.0) by specifying its nonzero entries. +val sv2: Vector = Vectors.sparse(3, Seq((0, 1.0), (2, 3.0))) +{% endhighlight %} + +***Note*** + +Scala imports `scala.collection.immutable.Vector` by default, so you have to import +`org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. + +
+ +
+ +The base class of local vectors is +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector), and we provide two +implementations: [`DenseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseVector) and +[`SparseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.SparseVector). We recommend +using the factory methods implemented in +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) to create local vectors. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; + +// Create a dense vector (1.0, 0.0, 3.0). +Vector dv = Vectors.dense(1.0, 0.0, 3.0); +// Create a sparse vector (1.0, 0.0, 3.0) by specifying its indices and values corresponding to nonzero entries. +Vector sv = Vectors.sparse(3, new int[] {0, 2}, new double[] {1.0, 3.0}); +{% endhighlight %} +
+ +
+MLlib recognizes the following types as dense vectors: + +* NumPy's [`array`](http://docs.scipy.org/doc/numpy/reference/generated/numpy.array.html) +* Python's list, e.g., `[1, 2, 3]` + +and the following as sparse vectors: + +* MLlib's [`SparseVector`](api/pyspark/pyspark.mllib.linalg.SparseVector-class.html). +* SciPy's + [`csc_matrix`](http://docs.scipy.org/doc/scipy/reference/generated/scipy.sparse.csc_matrix.html#scipy.sparse.csc_matrix) + with a single column + +We recommend using NumPy arrays over lists for efficiency, and using the factory methods implemented +in [`Vectors`](api/pyspark/pyspark.mllib.linalg.Vectors-class.html) to create sparse vectors. + +{% highlight python %} +import numpy as np +import scipy.sparse as sps +from pyspark.mllib.linalg import Vectors + +# Use a NumPy array as a dense vector. +dv1 = np.array([1.0, 0.0, 3.0]) +# Use a Python list as a dense vector. +dv2 = [1.0, 0.0, 3.0] +# Create a SparseVector. +sv1 = Vectors.sparse(3, [0, 2], [1.0, 3.0]) +# Use a single-column SciPy csc_matrix as a sparse vector. +sv2 = sps.csc_matrix((np.array([1.0, 3.0]), np.array([0, 2]), np.array([0, 2])), shape = (3, 1)) +{% endhighlight %} + +
+
+ +## Labeled point + +A labeled point is a local vector, either dense or sparse, associated with a label/response. +In MLlib, labeled points are used in supervised learning algorithms. +We use a double to store a label, so we can use labeled points in both regression and classification. +For binary classification, label should be either $0$ (negative) or $1$ (positive). +For multiclass classification, labels should be class indices staring from zero: $0, 1, 2, \ldots$. + +
+ +
+ +A labeled point is represented by the case class +[`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint). + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint + +// Create a labeled point with a positive label and a dense feature vector. +val pos = LabeledPoint(1.0, Vectors.dense(1.0, 0.0, 3.0)) + +// Create a labeled point with a negative label and a sparse feature vector. +val neg = LabeledPoint(0.0, Vectors.sparse(3, Array(0, 2), Array(1.0, 3.0))) +{% endhighlight %} +
+ +
+ +A labeled point is represented by +[`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint). + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; + +// Create a labeled point with a positive label and a dense feature vector. +LabeledPoint pos = new LabeledPoint(1.0, Vectors.dense(1.0, 0.0, 3.0)); + +// Create a labeled point with a negative label and a sparse feature vector. +LabeledPoint neg = new LabeledPoint(1.0, Vectors.sparse(3, new int[] {0, 2}, new double[] {1.0, 3.0})); +{% endhighlight %} +
+ +
+ +A labeled point is represented by +[`LabeledPoint`](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html). + +{% highlight python %} +from pyspark.mllib.linalg import SparseVector +from pyspark.mllib.regression import LabeledPoint + +# Create a labeled point with a positive label and a dense feature vector. +pos = LabeledPoint(1.0, [1.0, 0.0, 3.0]) + +# Create a labeled point with a negative label and a sparse feature vector. +neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0])) +{% endhighlight %} +
+
+ +***Sparse data*** + +It is very common in practice to have sparse training data. MLlib supports reading training +examples stored in `LIBSVM` format, which is the default format used by +[`LIBSVM`](http://www.csie.ntu.edu.tw/~cjlin/libsvm/) and +[`LIBLINEAR`](http://www.csie.ntu.edu.tw/~cjlin/liblinear/). It is a text format. Each line +represents a labeled sparse feature vector using the following format: + +~~~ +label index1:value1 index2:value2 ... +~~~ + +where the indices are one-based and in ascending order. +After loading, the feature indices are converted to zero-based. + +
+
+ +[`MLUtils.loadLibSVMData`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +examples stored in LIBSVM format. + +{% highlight scala %} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD + +val training: RDD[LabeledPoint] = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +{% endhighlight %} +
+ +
+[`MLUtils.loadLibSVMData`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +examples stored in LIBSVM format. + +{% highlight java %} +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDDimport; + +RDD[LabeledPoint] training = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +{% endhighlight %} +
+
+ +## Local matrix + +A local matrix has integer-typed row and column indices and double-typed values, stored on a single +machine. MLlib supports dense matrix, whose entry values are stored in a single double array in +column major. For example, the following matrix `\[ \begin{pmatrix} +1.0 & 2.0 \\ +3.0 & 4.0 \\ +5.0 & 6.0 +\end{pmatrix} +\]` +is stored in a one-dimensional array `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]` with the matrix size `(3, 2)`. +We are going to add sparse matrix in the next release. + +
+
+ +The base class of local matrices is +[`Matrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one +implementation: [`DenseMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseMatrix). +Sparse matrix will be added in the next release. We recommend using the factory methods implemented +in [`Matrices`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrices) to create local +matrices. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.{Matrix, Matrices} + +// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +val dm: Matrix = Matrices.dense(3, 2, Array(1.0, 3.0, 5.0, 2.0, 4.0, 6.0)) +{% endhighlight %} +
+ +
+ +The base class of local matrices is +[`Matrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one +implementation: [`DenseMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseMatrix). +Sparse matrix will be added in the next release. We recommend using the factory methods implemented +in [`Matrices`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrices) to create local +matrices. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Matrices; + +// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +Matrix dm = Matrices.dense(3, 2, new double[] {1.0, 3.0, 5.0, 2.0, 4.0, 6.0}); +{% endhighlight %} +
+ +
+ +## Distributed matrix + +A distributed matrix has long-typed row and column indices and double-typed values, stored +distributively in one or more RDDs. It is very important to choose the right format to store large +and distributed matrices. Converting a distributed matrix to a different format may require a +global shuffle, which is quite expensive. We implemented three types of distributed matrices in +this release and will add more types in the future. + +***Note*** + +The underlying RDDs of a distributed matrix must be deterministic, because we cache the matrix size. +It is always error-prone to have non-deterministic RDDs. + +### RowMatrix + +A `RowMatrix` is a row-oriented distributed matrix without meaningful row indices, backed by an RDD +of its rows, where each row is a local vector. This is similar to `data matrix` in the context of +multivariate statistics. Since each row is represented by a local vector, the number of columns is +limited by the integer range but it should be much smaller in practice. + +
+
+ +A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be +created from an `RDD[Vector]` instance. Then we can compute its column summary statistics. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.distributed.RowMatrix + +val rows: RDD[Vector] = ... // an RDD of local vectors +// Create a RowMatrix from an RDD[Vector]. +val mat: RowMatrix = new RowMatrix(rows) + +// Get its size. +val m = mat.numRows() +val n = mat.numCols() +{% endhighlight %} +
+ +
+ +A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be +created from a `JavaRDD` instance. Then we can compute its column summary statistics. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; + +JavaRDD rows = ... // a JavaRDD of local vectors +// Create a RowMatrix from an JavaRDD. +RowMatrix mat = new RowMatrix(rows.rdd()); + +// Get its size. +long m = mat.numRows(); +long n = mat.numCols(); +{% endhighlight %} +
+
+ +#### Multivariate summary statistics + +We provide column summary statistics for `RowMatrix`. +If the number of columns is not large, say, smaller than 3000, you can also compute +the covariance matrix as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the +number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, +which could be faster if the rows are sparse. + +
+
+ +`RowMatrix#computeColumnSummaryStatistics` returns an instance of +[`MultivariateStatisticalSummary`](api/mllib/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary + +val mat: RowMatrix = ... // a RowMatrix + +// Compute column summary statistics. +val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() +println(summary.mean) // a dense vector containing the mean value for each column +println(summary.variance) // column-wise variance +println(summary.numNonzers) // number of nonzeros in each column + +// Compute the covariance matrix. +val Cov: Matrix = mat.computeCovariance() +{% endhighlight %} +
+
+ +### IndexedRowMatrix + +An `IndexedRowMatrix` is similar to a `RowMatrix` but with meaningful row indices. It is backed by +an RDD of indexed rows, which each row is represented by its index (long-typed) and a local vector. + +
+
+ +An +[`IndexedRowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix) +can be created from an `RDD[IndexedRow]` instance, where +[`IndexedRow`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRow) is a +wrapper over `(Long, Vector)`. An `IndexedRowMatrix` can be converted to a `RowMatrix` by dropping +its row indices. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.distributed.{IndexedRow, IndexedRowMatrix, RowMatrix} + +val rows: RDD[IndexedRow] = ... // an RDD of indexed rows +// Create an IndexedRowMatrix from an RDD[IndexedRow]. +val mat: IndexedRowMatrix = new IndexedRowMatrix(rows) + +// Get its size. +val m = mat.numRows() +val n = mat.numCols() + +// Drop its row indices. +val rowMat: RowMatrix = mat.toRowMatrix() +{% endhighlight %} +
+ +
+ +An +[`IndexedRowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix) +can be created from an `JavaRDD` instance, where +[`IndexedRow`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRow) is a +wrapper over `(long, Vector)`. An `IndexedRowMatrix` can be converted to a `RowMatrix` by dropping +its row indices. + +{% highlight java %} +import org.apache.spark.mllib.linalg.distributed.IndexedRow; +import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; + +JavaRDD[IndexedRow] rows = ... // a JavaRDD of indexed rows +// Create an IndexedRowMatrix from a JavaRDD. +IndexedRowMatrix mat = new IndexedRowMatrix(rows.rdd()); + +// Get its size. +long m = mat.numRows(); +long n = mat.numCols(); + +// Drop its row indices. +RowMatrix rowMat = mat.toRowMatrix(); +{% endhighlight %} +
+ +### CoordinateMatrix + +A `CoordinateMatrix` is a distributed matrix backed by an RDD of its entries. Each entry is a tuple +of `(i: Long, j: Long, value: Double)`, where `i` is the row index, `j` is the column index, and +`value` is the entry value. A `CoordinateMatrix` should be used only in the case when both +dimensions of the matrix are huge and the matrix is very sparse. + +
+
+ +A +[`CoordinateMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) +can be created from an `RDD[MatrixEntry]` instance, where +[`MatrixEntry`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a +wrapper over `(Long, Long, Double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. In this release, we do not provide other +computation for `CoordinateMatrix`. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.distributed.{CoordinateMatrix, MatrixEntry} + +val entries: RDD[MatrixEntry] = ... // an RDD of matrix entries +// Create a CoordinateMatrix from an RDD[MatrixEntry]. +val mat: CoordinateMatrix = new CoordinateMatrix(entries) + +// Get its size. +val m = mat.numRows() +val n = mat.numCols() + +// Convert it to an IndexRowMatrix whose rows are sparse vectors. +val indexedRowMatrix = mat.toIndexedRowMatrix() +{% endhighlight %} +
+ +
+ +A +[`CoordinateMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) +can be created from a `JavaRDD` instance, where +[`MatrixEntry`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a +wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.distributed.CoordinateMatrix; +import org.apache.spark.mllib.linalg.distributed.MatrixEntry; + +JavaRDD entries = ... // a JavaRDD of matrix entries +// Create a CoordinateMatrix from a JavaRDD. +CoordinateMatrix mat = new CoordinateMatrix(entries); + +// Get its size. +long m = mat.numRows(); +long n = mat.numCols(); + +// Convert it to an IndexRowMatrix whose rows are sparse vectors. +IndexedRowMatrix indexedRowMatrix = mat.toIndexedRowMatrix(); +{% endhighlight %} +
+
diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md deleted file mode 100644 index b06e799577fb4..0000000000000 --- a/docs/mllib-classification-regression.md +++ /dev/null @@ -1,565 +0,0 @@ ---- -layout: global -title: MLlib - Classification and Regression ---- - -* Table of contents -{:toc} - - -`\[ -\newcommand{\R}{\mathbb{R}} -\newcommand{\E}{\mathbb{E}} -\newcommand{\x}{\mathbf{x}} -\newcommand{\y}{\mathbf{y}} -\newcommand{\wv}{\mathbf{w}} -\newcommand{\av}{\mathbf{\alpha}} -\newcommand{\bv}{\mathbf{b}} -\newcommand{\N}{\mathbb{N}} -\newcommand{\id}{\mathbf{I}} -\newcommand{\ind}{\mathbf{1}} -\newcommand{\0}{\mathbf{0}} -\newcommand{\unit}{\mathbf{e}} -\newcommand{\one}{\mathbf{1}} -\newcommand{\zero}{\mathbf{0}} -\]` - - -# Supervised Machine Learning -Supervised machine learning is the setting where we are given a set of training data examples -`$\{\x_i\}$`, each example `$\x_i$` coming with a corresponding label `$y_i$`. -Given the training data `$\{(\x_i,y_i)\}$`, we want to learn a function to predict these labels. -The two most well known classes of methods are -[classification](http://en.wikipedia.org/wiki/Statistical_classification), and -[regression](http://en.wikipedia.org/wiki/Regression_analysis). -In classification, the label is a category (e.g. whether or not emails are spam), whereas in -regression, the label is real value, and we want our prediction to be as close to the true value -as possible. - -Supervised Learning involves executing a learning *Algorithm* on a set of *labeled* training -examples. The algorithm returns a trained *Model* (such as for example a linear function) that -can predict the label for new data examples for which the label is unknown. - -## Discriminative Training using Linear Methods - -### Mathematical Formulation -Many standard *machine learning* methods can be formulated as a convex optimization problem, i.e. -the task of finding a minimizer of a convex function `$f$` that depends on a variable vector -`$\wv$` (called `weights` in the code), which has `$d$` entries. -Formally, we can write this as the optimization problem `$\min_{\wv \in\R^d} \; f(\wv)$`, where -the objective function is of the form -`\begin{equation} - f(\wv) := - \lambda\, R(\wv) + - \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) - \label{eq:regPrimal} - \ . -\end{equation}` -Here the vectors `$\x_i\in\R^d$` are the training data examples, for `$1\le i\le n$`, and -`$y_i\in\R$` are their corresponding labels, which we want to predict. - -The objective function `$f$` has two parts: -The *loss-function* measures the error of the model on the training data. The loss-function -`$L(\wv;.)$` must be a convex function in `$\wv$`. -The purpose of the [regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to -encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid -over-fitting. -Usually, the regularizer `$R(.)$` is chosen as either the standard (Euclidean) L2-norm, `$R(\wv) -:= \frac{1}{2}\|\wv\|^2$`, or the L1-norm, `$R(\wv) := \|\wv\|_1$`, see -[below](#using-different-regularizers) for more details. - -The fixed regularization parameter `$\lambda\ge0$` (`regParam` in the code) defines the trade-off -between the two goals of small loss and small model complexity. - - -### Binary Classification - -**Input:** Datapoints `$\x_i\in\R^{d}$`, labels `$y_i\in\{+1,-1\}$`, for `$1\le i\le n$`. - -**Distributed Datasets.** -For all currently implemented optimization methods for classification, the data must be -distributed between processes on the worker machines *by examples*. Machines hold consecutive -blocks of the `$n$` example/label pairs `$(\x_i,y_i)$`. -In other words, the input distributed dataset -([RDD](scala-programming-guide.html#resilient-distributed-datasets-rdds)) must be the set of -vectors `$\x_i\in\R^d$`. - -#### Support Vector Machine -The linear [Support Vector Machine (SVM)](http://en.wikipedia.org/wiki/Support_vector_machine) -has become a standard choice for classification tasks. -Here the loss function in formulation `$\eqref{eq:regPrimal}$` is given by the hinge-loss -`\[ -L(\wv;\x_i,y_i) := \max \{0, 1-y_i \wv^T \x_i \} \ . -\]` - -By default, SVMs are trained with an L2 regularization, which gives rise to the large-margin -interpretation if these classifiers. We also support alternative L1 regularization. In this case, -the primal optimization problem becomes an [LP](http://en.wikipedia.org/wiki/Linear_programming). - -#### Logistic Regression -Despite its name, [Logistic Regression](http://en.wikipedia.org/wiki/Logistic_regression) is a -binary classification method, again when the labels are given by binary values -`$y_i\in\{+1,-1\}$`. The logistic loss function in formulation `$\eqref{eq:regPrimal}$` is -defined as -`\[ -L(\wv;\x_i,y_i) := \log(1+\exp( -y_i \wv^T \x_i)) \ . -\]` - - -### Linear Regression (Least Squares, Lasso and Ridge Regression) - -**Input:** Data matrix `$A\in\R^{n\times d}$`, right hand side vector `$\y\in\R^n$`. - -**Distributed Datasets.** -For all currently implemented optimization methods for regression, the data matrix -`$A\in\R^{n\times d}$` must be distributed between the worker machines *by rows* of `$A$`. In -other words, the input distributed dataset -([RDD](scala-programming-guide.html#resilient-distributed-datasets-rdds)) must be the set of the -`$n$` rows `$A_{i:}$` of `$A$`. - -Least Squares Regression refers to the setting where we try to fit a vector `$\y\in\R^n$` by -linear combination of our observed data `$A\in\R^{n\times d}$`, which is given as a matrix. - -It comes in 3 flavors: - -#### Least Squares -Plain old [least squares](http://en.wikipedia.org/wiki/Least_squares) linear regression is the -problem of minimizing - `\[ f_{\text{LS}}(\wv) := \frac1n \|A\wv-\y\|_2^2 \ . \]` - -#### Lasso -The popular [Lasso](http://en.wikipedia.org/wiki/Lasso_(statistics)#Lasso_method) (alternatively -also known as `$L_1$`-regularized least squares regression) is given by - `\[ f_{\text{Lasso}}(\wv) := \frac1n \|A\wv-\y\|_2^2 + \lambda \|\wv\|_1 \ . \]` - -#### Ridge Regression -[Ridge regression](http://en.wikipedia.org/wiki/Ridge_regression) uses the same loss function but -with a L2 regularizer term: - `\[ f_{\text{Ridge}}(\wv) := \frac1n \|A\wv-\y\|_2^2 + \frac{\lambda}{2}\|\wv\|^2 \ . \]` - -**Loss Function.** -For all 3, the loss function (i.e. the measure of model fit) is given by the squared deviations -from the right hand side `$\y$`. -`\[ -\frac1n \|A\wv-\y\|_2^2 -= \frac1n \sum_{i=1}^n (A_{i:} \wv - y_i )^2 -= \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) -\]` -This is also known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). -In our generic problem formulation `$\eqref{eq:regPrimal}$`, this means the loss function is -`$L(\wv;\x_i,y_i) := (A_{i:} \wv - y_i )^2$`, each depending only on a single row `$A_{i:}$` of -the data matrix `$A$`. - - -### Using Different Regularizers - -As we have mentioned above, the purpose of *regularizer* in `$\eqref{eq:regPrimal}$` is to -encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid -over-fitting. -All machine learning methods for classification and regression that we have mentioned above are -of interest for different types of regularization, the 3 most common ones being - -* **L2-Regularization.** -`$R(\wv) := \frac{1}{2}\|\wv\|^2$`. -This regularizer is most commonly used for SVMs, logistic regression and ridge regression. - -* **L1-Regularization.** -`$R(\wv) := \|\wv\|_1$`. The L1 norm `$\|\wv\|_1$` is the sum of the absolut values of the -entries of a vector `$\wv$`. -This regularizer is most commonly used for sparse methods, and feature selection, such as the -Lasso. - -* **Non-Regularized.** -`$R(\wv):=0$`. -Of course we can also train the models without any regularization, or equivalently by setting the -regularization parameter `$\lambda:=0$`. - -The optimization problems of the form `$\eqref{eq:regPrimal}$` with convex regularizers such as -the 3 mentioned here can be conveniently optimized with gradient descent type methods (such as -SGD) which is implemented in `MLlib` currently, and explained in the next section. - - -### Optimization Methods Working on the Primal Formulation - -**Stochastic subGradient Descent (SGD).** -For optimization objectives `$f$` written as a sum, *stochastic subgradient descent (SGD)* can be -an efficient choice of optimization method, as we describe in the optimization section in more detail. -Because all methods considered here fit into the optimization formulation -`$\eqref{eq:regPrimal}$`, this is especially natural, because the loss is written as an average -of the individual losses coming from each datapoint. - -Picking one datapoint `$i\in[1..n]$` uniformly at random, we obtain a stochastic subgradient of -`$\eqref{eq:regPrimal}$`, with respect to `$\wv$` as follows: -`\[ -f'_{\wv,i} := L'_{\wv,i} + \lambda\, R'_\wv \ , -\]` -where `$L'_{\wv,i} \in \R^d$` is a subgradient of the part of the loss function determined by the -`$i$`-th datapoint, that is `$L'_{\wv,i} \in \frac{\partial}{\partial \wv} L(\wv;\x_i,y_i)$`. -Furthermore, `$R'_\wv$` is a subgradient of the regularizer `$R(\wv)$`, i.e. `$R'_\wv \in -\frac{\partial}{\partial \wv} R(\wv)$`. The term `$R'_\wv$` does not depend on which random -datapoint is picked. - - - -**Gradients.** -The following table summarizes the gradients (or subgradients) of all loss functions and -regularizers that we currently support: - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
FunctionStochastic (Sub)Gradient
SVM Hinge Loss$L(\wv;\x_i,y_i) := \max \{0, 1-y_i \wv^T \x_i \}$$L'_{\wv,i} = \begin{cases}-y_i \x_i & \text{if $y_i \wv^T \x_i <1$}, \\ 0 & -\text{otherwise}.\end{cases}$
Logistic Loss$L(\wv;\x_i,y_i) := \log(1+\exp( -y_i \wv^T \x_i))$$L'_{\wv,i} = -y_i \x_i \left(1-\frac1{1+\exp(-y_i \wv^T \x_i)} \right)$
Least Squares Loss$L(\wv;\x_i,y_i) := (A_{i:} \wv - y_i)^2$$L'_{\wv,i} = 2 A_{i:}^T (A_{i:} \wv - y_i)$
Non-Regularized$R(\wv) := 0$$R'_\wv = \0$
L2 Regularizer$R(\wv) := \frac{1}{2}\|\wv\|^2$$R'_\wv = \wv$
L1 Regularizer$R(\wv) := \|\wv\|_1$$R'_\wv = \mathop{sign}(\wv)$
- -Here `$\mathop{sign}(\wv)$` is the vector consisting of the signs (`$\pm1$`) of all the entries -of `$\wv$`. -Also, note that `$A_{i:} \in \R^d$` is a row-vector, but the gradient is a column vector. - -## Decision Tree Classification and Regression - -Decision trees and their ensembles are popular methods for the machine learning tasks of classification and regression. Decision trees are widely used since they are easy to interpret, handle categorical variables, extend to the multi-class classification setting, do not require feature scaling and are able to capture non-linearities and feature interactions. Tree ensemble algorithms such as decision forest and boosting are among the top performers for classification and regression tasks. - -### Basic Algorithm - -The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature space by choosing a single element from the *best split set* where each element of the set maximimizes the information gain at a tree node. In other words, the split chosen at each tree node is chosen from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` is the information gain when a split `$s$` is applied to a dataset `$D$`. - -#### Node Impurity and Information Gain - -The *node impurity* is a measure of the homogeneity of the labels at the node. The current implementation provides two impurity measures for classification (Gini index and entropy) and one impurity measure for regression (variance). - - - - - - - - - - - - - - - - -
ImpurityTaskFormulaDescription
Gini indexClassification$\sum_{i=1}^{M} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
EntropyClassification$\sum_{i=1}^{M} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
VarianceClassification$\frac{1}{n} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^n x_i$.
- -The *information gain* is the difference in the parent node impurity and the weighted sum of the two child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` into two datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, respectively: - -`$IG(D,s) = Impurity(D) - \frac{N_{left}}{N} Impurity(D_{left}) - \frac{N_{right}}{N} Impurity(D_{right})$` - -#### Split Candidates - -**Continuous Features** - -For small datasets in single machine implementations, the split candidates for each continuous feature are typically the unique values for the feature. Some implementations sort the feature values and then use the ordered unique values as split candidates for faster tree calculations. - -Finding ordered unique feature values is computationally intensive for large distributed datasets. One can get an approximate set of split candidates by performing a quantile calculation over a sampled fraction of the data. The ordered splits create "bins" and the maximum number of such bins can be specified using the `maxBins` parameters. - -Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of bins if the condition is not satisfied. - -**Categorical Features** - -For `$M$` categorical features, one could come up with `$2^M-1$` split candidates. However, for binary classification, the number of split candidates can be reduced to `$M-1$` by ordering the categorical feature values by the proportion of labels falling in one of the two classes (see Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for details). For example, for a binary classification problem with one categorical feature with three categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical features are orded as A followed by C followed B or A, B, C. The two split candidates are A \| C, B and A , B \| C where \| denotes the split. - -#### Stopping Rule - -The recursive tree construction is stopped at a node when one of the two conditions is met: - -1. The node depth is equal to the `maxDepth` training paramemter -2. No split candidate leads to an information gain at the node. - -### Implementation Details - -The tree implementation stores an Array[Double] of size *O(#features \* #splits \* 2^maxDepth)* in memory for aggregating histograms over partitions. Based upon the 'maxMemory' parameter set during training (default is 128 MB), the task is broken down into smaller groups to avoid out-of-memory errors during computation. - -## Implementation in MLlib - -#### Linear Methods - -For both classification and regression algorithms with convex loss functions, `MLlib` implements a simple distributed version of -stochastic subgradient descent (SGD), building on the underlying gradient descent primitive (as -described in the -optimization section). -All provided algorithms take as input a regularization parameter (`regParam`) along with various -parameters associated with stochastic gradient -descent (`stepSize`, `numIterations`, `miniBatchFraction`). -For each of them, we support all 3 possible regularizations (none, L1 or L2). - -Available algorithms for binary classification: - -* [SVMWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.SVMWithSGD) -* [LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) - -Available algorithms for linear regression: - -* [LinearRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) -* [RidgeRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -* [LassoWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD) - -Behind the scenes, all above methods use the SGD implementation from the -gradient descent primitive in MLlib, see the -optimization part: - -* [GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) - -#### Tree-based Methods - -The decision tree algorithm supports binary classification and regression: - -* [DecisionTee](api/mllib/index.html#org.apache.spark.mllib.tree.DecisionTree) - - -# Usage in Scala - -Following code snippets can be executed in `spark-shell`. - -## Linear Methods - - -#### Binary Classification - -The following code snippet illustrates how to load a sample dataset, execute a -training algorithm on this training data using a static method in the algorithm -object, and make predictions with the resulting model to compute the training -error. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.classification.SVMWithSGD -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_svm_data.txt") -val parsedData = data.map { line => - val parts = line.split(' ').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} - -// Run training algorithm to build the model -val numIterations = 100 -val model = SVMWithSGD.train(parsedData, numIterations) - -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count -println("Training Error = " + trainErr) -{% endhighlight %} - - -The `SVMWithSGD.train()` method by default performs L2 regularization with the -regularization parameter set to 1.0. If we want to configure this algorithm, we -can customize `SVMWithSGD` further by creating a new object directly and -calling setter methods. All other MLlib algorithms support customization in -this way as well. For example, the following code produces an L1 regularized -variant of SVMs with regularization parameter set to 0.1, and runs the training -algorithm for 200 iterations. - -{% highlight scala %} -import org.apache.spark.mllib.optimization.L1Updater - -val svmAlg = new SVMWithSGD() -svmAlg.optimizer.setNumIterations(200) - .setRegParam(0.1) - .setUpdater(new L1Updater) -val modelL1 = svmAlg.run(parsedData) -{% endhighlight %} - -#### Linear Regression - -The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. -The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -{% highlight scala %} -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data -val data = sc.textFile("mllib/data/ridge-data/lpsa.data") -val parsedData = data.map { line => - val parts = line.split(',') - LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) -} - -// Building the model -val numIterations = 100 -val model = LinearRegressionWithSGD.train(parsedData, numIterations) - -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.reduce(_ + _) / valuesAndPreds.count -println("training Mean Squared Error = " + MSE) -{% endhighlight %} - - -Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training -[Mean Squared Errors](http://en.wikipedia.org/wiki/Mean_squared_error). - -## Decision Tree - -#### Classification - -The example below demonstrates how to load a CSV file, parse it as an RDD of LabeledPoint and then perform classification using a decision tree using Gini index as an impurity measure and a maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Gini - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} - -// Run training algorithm to build the model -val maxDepth = 5 -val model = DecisionTree.train(parsedData, Classification, Gini, maxDepth) - -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count -println("Training Error = " + trainErr) -{% endhighlight %} - -#### Regression - -The example below demonstrates how to load a CSV file, parse it as an RDD of LabeledPoint and then perform regression using a decision tree using variance as an impurity measure and a maximum tree depth of 5. The Mean Squared Error is computed at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Variance - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} - -// Run training algorithm to build the model -val maxDepth = 5 -val model = DecisionTree.train(parsedData, Regression, Variance, maxDepth) - -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count -println("training Mean Squared Error = " + MSE) -{% endhighlight %} - - -# Usage in Java - -All of MLlib's methods use Java-friendly types, so you can import and call them there the same -way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the -Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. - -# Usage in Python - -Following examples can be tested in the PySpark shell. - -## Linear Methods - -### Binary Classification -The following example shows how to load a sample dataset, build Logistic Regression model, -and make predictions with the resulting model to compute the training error. - -{% highlight python %} -from pyspark.mllib.classification import LogisticRegressionWithSGD -from pyspark.mllib.regression import LabeledPoint -from numpy import array - -# Load and parse the data -def parsePoint(line): - values = [float(x) for x in line.split(' ')] - return LabeledPoint(values[0], values[1:]) - -data = sc.textFile("mllib/data/sample_svm_data.txt") -parsedData = data.map(parsePoint) - -# Build the model -model = LogisticRegressionWithSGD.train(parsedData) - -# Evaluating the model on training data -labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) -trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) -print("Training Error = " + str(trainErr)) -{% endhighlight %} - -### Linear Regression -The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. -The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -{% highlight python %} -from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD -from numpy import array - -# Load and parse the data -def parsePoint(line): - values = [float(x) for x in line.replace(',', ' ').split(' ')] - return LabeledPoint(values[0], values[1:]) - -data = sc.textFile("mllib/data/ridge-data/lpsa.data") -parsedData = data.map(parsePoint) - -# Build the model -model = LinearRegressionWithSGD.train(parsedData) - -# Evaluate the model on training data -valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) -MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y) / valuesAndPreds.count() -print("Mean Squared Error = " + str(MSE)) -{% endhighlight %} diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 50a8671560737..b3293afe408d0 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -1,19 +1,21 @@ --- layout: global -title: MLlib - Clustering +title: MLlib - Clustering --- * Table of contents {:toc} -# Clustering +## Clustering Clustering is an unsupervised learning problem whereby we aim to group subsets of entities with one another based on some notion of similarity. Clustering is often used for exploratory analysis and/or as a component of a hierarchical supervised learning pipeline (in which distinct classifiers or regression -models are trained for each cluster). MLlib supports +models are trained for each cluster). + +MLlib supports [k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of the most commonly used clustering algorithms that clusters the data points into predfined number of clusters. The MLlib implementation includes a parallelized @@ -31,17 +33,14 @@ a given dataset, the algorithm returns the best clustering result). * *initializiationSteps* determines the number of steps in the k-means\|\| algorithm. * *epsilon* determines the distance threshold within which we consider k-means to have converged. -Available algorithms for clustering: - -* [KMeans](api/mllib/index.html#org.apache.spark.mllib.clustering.KMeans) - - - -# Usage in Scala +## Examples +
+
Following code snippets can be executed in `spark-shell`. -In the following example after loading and parsing data, we use the KMeans object to cluster the data +In the following example after loading and parsing data, we use the +[`KMeans`](api/mllib/index.html#org.apache.spark.mllib.clustering.KMeans) object to cluster the data into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the optimal *k* is usually one where there is an "elbow" in the WSSSE graph. @@ -63,22 +62,22 @@ val clusters = KMeans.train(parsedData, numClusters, numIterations) val WSSSE = clusters.computeCost(parsedData) println("Within Set Sum of Squared Errors = " + WSSSE) {% endhighlight %} +
- -# Usage in Java - +
All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. +
-# Usage in Python +
Following examples can be tested in the PySpark shell. -In the following example after loading and parsing data, we use the KMeans object to cluster the data -into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within -Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the -optimal *k* is usually one where there is an "elbow" in the WSSSE graph. +In the following example after loading and parsing data, we use the KMeans object to cluster the +data into two clusters. The number of desired clusters is passed to the algorithm. We then compute +Within Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In +fact the optimal *k* is usually one where there is an "elbow" in the WSSSE graph. {% highlight python %} from pyspark.mllib.clustering import KMeans @@ -91,7 +90,7 @@ parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) # Build the model (cluster the data) clusters = KMeans.train(parsedData, 2, maxIterations=10, - runs=10, initialization_mode="random") + runs=10, initializationMode="random") # Evaluate clustering by computing Within Set Sum of Squared Errors def error(point): @@ -101,7 +100,6 @@ def error(point): WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y) print("Within Set Sum of Squared Error = " + str(WSSSE)) {% endhighlight %} +
-Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training Mean Squared -Errors. - +
diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index aa22f67b303ce..79f5e3a7ca4fb 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -1,12 +1,12 @@ --- layout: global -title: MLlib - Collaborative Filtering +title: MLlib - Collaborative Filtering --- * Table of contents {:toc} -# Collaborative Filtering +## Collaborative filtering [Collaborative filtering](http://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering) is commonly used for recommender systems. These techniques aim to fill in the @@ -14,44 +14,43 @@ missing entries of a user-item association matrix. MLlib currently supports model-based collaborative filtering, in which users and products are described by a small set of latent factors that can be used to predict missing entries. In particular, we implement the [alternating least squares -(ALS)](http://www2.research.att.com/~volinsky/papers/ieeecomputer.pdf) +(ALS)](http://dl.acm.org/citation.cfm?id=1608614) algorithm to learn these latent factors. The implementation in MLlib has the following parameters: -* *numBlocks* is the number of blacks used to parallelize computation (set to -1 to auto-configure). +* *numBlocks* is the number of blocks used to parallelize computation (set to -1 to auto-configure). * *rank* is the number of latent factors in our model. * *iterations* is the number of iterations to run. * *lambda* specifies the regularization parameter in ALS. -* *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for *implicit feedback* data -* *alpha* is a parameter applicable to the implicit feedback variant of ALS that governs the *baseline* confidence in preference observations +* *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for + *implicit feedback* data. +* *alpha* is a parameter applicable to the implicit feedback variant of ALS that governs the + *baseline* confidence in preference observations. -## Explicit vs Implicit Feedback +### Explicit vs. implicit feedback The standard approach to matrix factorization based collaborative filtering treats the entries in the user-item matrix as *explicit* preferences given by the user to the item. -It is common in many real-world use cases to only have access to *implicit feedback* -(e.g. views, clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with -such data is taken from -[Collaborative Filtering for Implicit Feedback Datasets](http://www2.research.att.com/~yifanhu/PUB/cf.pdf). -Essentially instead of trying to model the matrix of ratings directly, this approach treats the data as -a combination of binary preferences and *confidence values*. The ratings are then related -to the level of confidence in observed user preferences, rather than explicit ratings given to items. -The model then tries to find latent factors that can be used to predict the expected preference of a user -for an item. +It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, +clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with such data is taken +from +[Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). +Essentially instead of trying to model the matrix of ratings directly, this approach treats the data +as a combination of binary preferences and *confidence values*. The ratings are then related to the +level of confidence in observed user preferences, rather than explicit ratings given to items. The +model then tries to find latent factors that can be used to predict the expected preference of a +user for an item. -Available algorithms for collaborative filtering: +## Examples -* [ALS](api/mllib/index.html#org.apache.spark.mllib.recommendation.ALS) - - -# Usage in Scala - -Following code snippets can be executed in `spark-shell`. +
+
In the following example we load rating data. Each row consists of a user, a product and a rating. -We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation -model by measuring the Mean Squared Error of rating prediction. +We use the default [ALS.train()](api/mllib/index.html#org.apache.spark.mllib.recommendation.ALS$) +method which assumes ratings are explicit. We evaluate the +recommendation model by measuring the Mean Squared Error of rating prediction. {% highlight scala %} import org.apache.spark.mllib.recommendation.ALS @@ -64,8 +63,9 @@ val ratings = data.map(_.split(',') match { }) // Build the recommendation model using ALS +val rank = 10 val numIterations = 20 -val model = ALS.train(ratings, 1, 20, 0.01) +val model = ALS.train(ratings, rank, numIterations, 0.01) // Evaluate the model on rating data val usersProducts = ratings.map{ case Rating(user, product, rate) => (user, product)} @@ -85,19 +85,19 @@ If the rating matrix is derived from other source of information (i.e., it is in other signals), you can use the trainImplicit method to get better results. {% highlight scala %} -val model = ALS.trainImplicit(ratings, 1, 20, 0.01) +val alpha = 0.01 +val model = ALS.trainImplicit(ratings, rank, numIterations, alpha) {% endhighlight %} +
-# Usage in Java - +
All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. +
-# Usage in Python -Following examples can be tested in the PySpark shell. - +
In the following example we load rating data. Each row consists of a user, a product and a rating. We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation by measuring the Mean Squared Error of rating prediction. @@ -111,7 +111,9 @@ data = sc.textFile("mllib/data/als/test.data") ratings = data.map(lambda line: array([float(x) for x in line.split(',')])) # Build the recommendation model using Alternating Least Squares -model = ALS.train(ratings, 1, 20) +rank = 10 +numIterations = 20 +model = ALS.train(ratings, rank, numIterations) # Evaluate the model on training data testdata = ratings.map(lambda p: (int(p[0]), int(p[1]))) @@ -126,5 +128,13 @@ signals), you can use the trainImplicit method to get better results. {% highlight python %} # Build the recommendation model using Alternating Least Squares based on implicit ratings -model = ALS.trainImplicit(ratings, 1, 20) +model = ALS.trainImplicit(ratings, rank, numIterations, alpha = 0.01) {% endhighlight %} +
+ +
+ +## Tutorial + +[AMP Camp](http://ampcamp.berkeley.edu/) provides a hands-on tutorial for +[personalized movie recommendation with MLlib](http://ampcamp.berkeley.edu/big-data-mini-course/movie-recommendation-with-mllib.html). diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md new file mode 100644 index 0000000000000..0693766990732 --- /dev/null +++ b/docs/mllib-decision-tree.md @@ -0,0 +1,185 @@ +--- +layout: global +title: MLlib - Decision Tree +--- + +* Table of contents +{:toc} + +Decision trees and their ensembles are popular methods for the machine learning tasks of +classification and regression. Decision trees are widely used since they are easy to interpret, +handle categorical variables, extend to the multiclass classification setting, do not require +feature scaling and are able to capture nonlinearities and feature interactions. Tree ensemble +algorithms such as decision forest and boosting are among the top performers for classification and +regression tasks. + +## Basic algorithm + +The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature +space by choosing a single element from the *best split set* where each element of the set maximizes +the information gain at a tree node. In other words, the split chosen at each tree node is chosen +from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` is the information +gain when a split `$s$` is applied to a dataset `$D$`. + +### Node impurity and information gain + +The *node impurity* is a measure of the homogeneity of the labels at the node. The current +implementation provides two impurity measures for classification (Gini impurity and entropy) and one +impurity measure for regression (variance). + + + + + + + + + + + + + + + + + + + + + + +
ImpurityTaskFormulaDescription
Gini impurityClassification$\sum_{i=1}^{M} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
EntropyClassification$\sum_{i=1}^{M} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
VarianceRegression$\frac{1}{n} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, + $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^n x_i$.
+ +The *information gain* is the difference in the parent node impurity and the weighted sum of the two +child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` into two +datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, respectively: + +`$IG(D,s) = Impurity(D) - \frac{N_{left}}{N} Impurity(D_{left}) - \frac{N_{right}}{N} Impurity(D_{right})$` + +### Split candidates + +**Continuous features** + +For small datasets in single machine implementations, the split candidates for each continuous +feature are typically the unique values for the feature. Some implementations sort the feature +values and then use the ordered unique values as split candidates for faster tree calculations. + +Finding ordered unique feature values is computationally intensive for large distributed +datasets. One can get an approximate set of split candidates by performing a quantile calculation +over a sampled fraction of the data. The ordered splits create "bins" and the maximum number of such +bins can be specified using the `maxBins` parameters. + +Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario +since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of +bins if the condition is not satisfied. + +**Categorical features** + +For `$M$` categorical features, one could come up with `$2^M-1$` split candidates. However, for +binary classification, the number of split candidates can be reduced to `$M-1$` by ordering the +categorical feature values by the proportion of labels falling in one of the two classes (see +Section 9.2.4 in +[Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for +details). For example, for a binary classification problem with one categorical feature with three +categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical +features are orded as A followed by C followed B or A, B, C. The two split candidates are A \| C, B +and A , B \| C where \| denotes the split. + +### Stopping rule + +The recursive tree construction is stopped at a node when one of the two conditions is met: + +1. The node depth is equal to the `maxDepth` training parammeter +2. No split candidate leads to an information gain at the node. + +### Practical limitations + +1. The tree implementation stores an Array[Double] of size *O(#features \* #splits \* 2^maxDepth)* + in memory for aggregating histograms over partitions. The current implementation might not scale + to very deep trees since the memory requirement grows exponentially with tree depth. +2. The implemented algorithm reads both sparse and dense data. However, it is not optimized for + sparse input. +3. Python is not supported in this release. + +We are planning to solve these problems in the near future. Please drop us a line if you encounter +any issues. + +## Examples + +### Classification + +The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then +perform classification using a decision tree using Gini impurity as an impurity measure and a +maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy. + +
+
+{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.impurity.Gini + +// Load and parse the data file +val data = sc.textFile("mllib/data/sample_tree_data.csv") +val parsedData = data.map { line => + val parts = line.split(',').map(_.toDouble) + LabeledPoint(parts(0), Vectors.dense(parts.tail)) +} + +// Run training algorithm to build the model +val maxDepth = 5 +val model = DecisionTree.train(parsedData, Classification, Gini, maxDepth) + +// Evaluate model on training examples and compute training error +val labelAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count +println("Training Error = " + trainErr) +{% endhighlight %} +
+
+ +### Regression + +The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then +perform regression using a decision tree using variance as an impurity measure and a maximum tree +depth of 5. The Mean Squared Error (MSE) is computed at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +
+
+{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.impurity.Variance + +// Load and parse the data file +val data = sc.textFile("mllib/data/sample_tree_data.csv") +val parsedData = data.map { line => + val parts = line.split(',').map(_.toDouble) + LabeledPoint(parts(0), Vectors.dense(parts.tail)) +} + +// Run training algorithm to build the model +val maxDepth = 5 +val model = DecisionTree.train(parsedData, Regression, Variance, maxDepth) + +// Evaluate model on training examples and compute training error +val valuesAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count +println("training Mean Squared Error = " + MSE) +{% endhighlight %} +
+
diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md new file mode 100644 index 0000000000000..4e9ecf7c006fa --- /dev/null +++ b/docs/mllib-dimensionality-reduction.md @@ -0,0 +1,86 @@ +--- +layout: global +title: MLlib - Dimensionality Reduction +--- + +* Table of contents +{:toc} + +[Dimensionality reduction](http://en.wikipedia.org/wiki/Dimensionality_reduction) is the process +of reducing the number of variables under consideration. +It is used to extract latent features from raw and noisy features, +or compress data while maintaining the structure. +In this release, we provide preliminary support for dimensionality reduction on tall-and-skinny matrices. + +## Singular value decomposition (SVD) + +[Singular value decomposition (SVD)](http://en.wikipedia.org/wiki/Singular_value_decomposition) +factorizes a matrix into three matrices: $U$, $\Sigma$, and $V$ such that + +`\[ +A = U \Sigma V^T, +\]` + +where + +* $U$ is an orthonormal matrix, whose columns are called left singular vectors, +* $\Sigma$ is a diagonal matrix with non-negative diagonals in descending order, + whose diagonals are called singular values, +* $V$ is an orthonormal matrix, whose columns are called right singular vectors. + +For large matrices, usually we don't need the complete factorization but only the top singular +values and its associated singular vectors. This can save storage, and more importantly, de-noise +and recover the low-rank structure of the matrix. + +If we keep the top $k$ singular values, then the dimensions of the return will be: + +* `$U$`: `$m \times k$`, +* `$\Sigma$`: `$k \times k$`, +* `$V$`: `$n \times k$`. + +In this release, we provide SVD computation to row-oriented matrices that have only a few columns, +say, less than $1000$, but many rows, which we call *tall-and-skinny*. + +
+
+{% highlight scala %} +val mat: RowMatrix = ... + +// Compute the top 20 singular values and corresponding singular vectors. +val svd: SingularValueDecomposition[RowMatrix, Matrix] = mat.computeSVD(20, computeU = true) +val U: RowMatrix = svd.U // The U factor is a RowMatrix. +val s: Vector = svd.s // The singular values are stored in a local dense vector. +val V: Matrix = svd.V // The V factor is a local dense matrix. +{% endhighlight %} +
+Same code applies to `IndexedRowMatrix`. +The only difference that the `U` matrix becomes an `IndexedRowMatrix`. +
+ +## Principal component analysis (PCA) + +[Principal component analysis (PCA)](http://en.wikipedia.org/wiki/Principal_component_analysis) is a +statistical method to find a rotation such that the first coordinate has the largest variance +possible, and each succeeding coordinate in turn has the largest variance possible. The columns of +the rotation matrix are called principal components. PCA is used widely in dimensionality reduction. + +In this release, we implement PCA for tall-and-skinny matrices stored in row-oriented format. + +
+
+ +The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +and use them to project the vectors into a low-dimensional space. +The number of columns should be small, e.g, less than 1000. + +{% highlight scala %} +val mat: RowMatrix = ... + +// Compute the top 10 principal components. +val pc: Matrix = mat.computePrincipalComponents(10) // Principal components are stored in a local dense matrix. + +// Project the rows to the linear space spanned by the top 10 principal components. +val projected: RowMatrix = mat.multiply(pc) +{% endhighlight %} +
+
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 4236b0c8b6c99..c49f857d07557 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -3,63 +3,121 @@ layout: global title: Machine Learning Library (MLlib) --- +MLlib is a Spark implementation of some common machine learning algorithms and utilities, +including classification, regression, clustering, collaborative +filtering, dimensionality reduction, as well as underlying optimization primitives: -MLlib is a Spark implementation of some common machine learning (ML) -functionality, as well associated tests and data generators. MLlib -currently supports four common types of machine learning problem settings, -namely classification, regression, clustering and collaborative filtering, -as well as an underlying gradient descent optimization primitive and several -linear algebra methods. - -# Available Methods -The following links provide a detailed explanation of the methods and usage examples for each of them: - -* Classification and Regression - * Binary Classification - * SVM (L1 and L2 regularized) - * Logistic Regression (L1 and L2 regularized) - * Linear Regression - * Least Squares - * Lasso - * Ridge Regression - * Decision Tree (for classification and regression) -* Clustering - * k-Means -* Collaborative Filtering - * Matrix Factorization using Alternating Least Squares -* Optimization - * Gradient Descent and Stochastic Gradient Descent -* Linear Algebra - * Singular Value Decomposition - * Principal Component Analysis - -# Data Types - -Most MLlib algorithms operate on RDDs containing vectors. In Java and Scala, the -[Vector](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) class is used to -represent vectors. You can create either dense or sparse vectors using the -[Vectors](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) factory. - -In Python, MLlib can take the following vector types: - -* [NumPy](http://www.numpy.org) arrays -* Standard Python lists (e.g. `[1, 2, 3]`) -* The MLlib [SparseVector](api/pyspark/pyspark.mllib.linalg.SparseVector-class.html) class -* [SciPy sparse matrices](http://docs.scipy.org/doc/scipy/reference/sparse.html) - -For efficiency, we recommend using NumPy arrays over lists, and using the -[CSC format](http://docs.scipy.org/doc/scipy/reference/generated/scipy.sparse.csc_matrix.html#scipy.sparse.csc_matrix) -for SciPy matrices, or MLlib's own SparseVector class. - -Several other simple data types are used throughout the library, e.g. the LabeledPoint -class ([Java/Scala](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint), -[Python](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html)) for labeled data. - -# Dependencies -MLlib uses the [jblas](https://github.com/mikiobraun/jblas) linear algebra library, which itself -depends on native Fortran routines. You may need to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) -if it is not already present on your nodes. MLlib will throw a linking error if it cannot -detect these libraries automatically. +* [Basics](mllib-basics.html) + * data types + * summary statistics +* Classification and regression + * [linear support vector machine (SVM)](mllib-linear-methods.html#linear-support-vector-machine-svm) + * [logistic regression](mllib-linear-methods.html#logistic-regression) + * [linear least squares, Lasso, and ridge regression](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression) + * [decision tree](mllib-decision-tree.html) + * [naive Bayes](mllib-naive-bayes.html) +* [Collaborative filtering](mllib-collaborative-filtering.html) + * alternating least squares (ALS) +* [Clustering](mllib-clustering.html) + * k-means +* [Dimensionality reduction](mllib-dimensionality-reduction.html) + * singular value decomposition (SVD) + * principal component analysis (PCA) +* [Optimization](mllib-optimization.html) + * stochastic gradient descent + * limited-memory BFGS (L-BFGS) + +MLlib is currently a *beta* component under active development. +The APIs may change in the future releases, and we will provide migration guide between releases. + +## Dependencies + +MLlib uses linear algebra packages [Breeze](http://www.scalanlp.org/), which depends on +[netlib-java](https://github.com/fommil/netlib-java), and +[jblas](https://github.com/mikiobraun/jblas). +`netlib-java` and `jblas` depend on native Fortran routines. +You need to install the +[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) if it is not +already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries +automatically. Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's +dependency set. If no native library is available at runtime, you will see a warning message. To +use native libraries from `netlib-java`, please include artifact +`com.github.fommil.netlib:all:1.1.2` as a dependency of your project or build your own (see +[instructions](https://github.com/fommil/netlib-java/blob/master/README.md#machine-optimised-system-libraries)). To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. + +--- + +## Migration guide + +### From 0.9 to 1.0 + +In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few +breaking changes. If your data is sparse, please store it in a sparse format instead of dense to +take advantage of sparsity in both storage and computation. + +
+
+ +We used to represent a feature vector by `Array[Double]`, which is replaced by +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used +to accept `RDD[Array[Double]]` now take +`RDD[Vector]`. [`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) +is now a wrapper of `(Double, Vector)` instead of `(Double, Array[Double])`. Converting +`Array[Double]` to `Vector` is straightforward: + +{% highlight scala %} +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +val array: Array[Double] = ... // a double array +val vector: Vector = Vectors.dense(array) // a dense vector +{% endhighlight %} + +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors. + +*Note*. Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. + +
+ +
+ +We used to represent a feature vector by `double[]`, which is replaced by +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used +to accept `RDD` now take +`RDD`. [`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) +is now a wrapper of `(double, Vector)` instead of `(double, double[])`. Converting `double[]` to +`Vector` is straightforward: + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; + +double[] array = ... // a double array +Vector vector = Vectors.dense(array) // a dense vector +{% endhighlight %} + +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to +create sparse vectors. + +
+ +
+ +We used to represent a labeled feature vector in a NumPy array, where the first entry corresponds to +the label and the rest are features. This representation is replaced by class +[`LabeledPoint`](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html), which takes both +dense and sparse feature vectors. + +{% highlight python %} +from pyspark.mllib.linalg import SparseVector +from pyspark.mllib.regression import LabeledPoint + +# Create a labeled point with a positive label and a dense feature vector. +pos = LabeledPoint(1.0, [1.0, 0.0, 3.0]) + +# Create a labeled point with a negative label and a sparse feature vector. +neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0])) +{% endhighlight %} +
+
diff --git a/docs/mllib-linear-algebra.md b/docs/mllib-linear-algebra.md deleted file mode 100644 index 09598be7903ac..0000000000000 --- a/docs/mllib-linear-algebra.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -layout: global -title: MLlib - Linear Algebra ---- - -* Table of contents -{:toc} - - -# Singular Value Decomposition -Singular Value `Decomposition` for Tall and Skinny matrices. -Given an `$m \times n$` matrix `$A$`, we can compute matrices `$U,S,V$` such that - -`\[ - A = U \cdot S \cdot V^T - \]` - -There is no restriction on m, but we require n^2 doubles to -fit in memory locally on one machine. -Further, n should be less than m. - -The decomposition is computed by first computing `$A^TA = V S^2 V^T$`, -computing SVD locally on that (since `$n \times n$` is small), -from which we recover `$S$` and `$V$`. -Then we compute U via easy matrix multiplication -as `$U = A \cdot V \cdot S^{-1}$`. - -Only singular vectors associated with largest k singular values -are recovered. If there are k -such values, then the dimensions of the return will be: - -* `$S$` is `$k \times k$` and diagonal, holding the singular values on diagonal. -* `$U$` is `$m \times k$` and satisfies `$U^T U = \mathop{eye}(k)$`. -* `$V$` is `$n \times k$` and satisfies `$V^T V = \mathop{eye}(k)$`. - -All input and output is expected in sparse matrix format, 0-indexed -as tuples of the form ((i,j),value) all in -SparseMatrix RDDs. Below is example usage. - -{% highlight scala %} - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.SVD -import org.apache.spark.mllib.linalg.SparseMatrix -import org.apache.spark.mllib.linalg.MatrixEntry - -// Load and parse the data file -val data = sc.textFile("mllib/data/als/test.data").map { line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble) -} -val m = 4 -val n = 4 -val k = 1 - -// recover largest singular vector -val decomposed = SVD.sparseSVD(SparseMatrix(data, m, n), k) -val = decomposed.S.data - -println("singular values = " + s.toArray.mkString) -{% endhighlight %} - - -# Principal Component Analysis - -Computes the top k principal component coefficients for the m-by-n data matrix X. -Rows of X correspond to observations and columns correspond to variables. -The coefficient matrix is n-by-k. Each column of the return matrix contains coefficients -for one principal component, and the columns are in descending -order of component variance. This function centers the data and uses the -singular value decomposition (SVD) algorithm. - -All input and output is expected in DenseMatrix matrix format. See the examples directory -under "SparkPCA.scala" for example usage. diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md new file mode 100644 index 0000000000000..8108440698495 --- /dev/null +++ b/docs/mllib-linear-methods.md @@ -0,0 +1,389 @@ +--- +layout: global +title: MLlib - Linear Methods +--- + +* Table of contents +{:toc} + + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +## Mathematical formulation + +Many standard *machine learning* methods can be formulated as a convex optimization problem, i.e. +the task of finding a minimizer of a convex function `$f$` that depends on a variable vector +`$\wv$` (called `weights` in the code), which has `$d$` entries. +Formally, we can write this as the optimization problem `$\min_{\wv \in\R^d} \; f(\wv)$`, where +the objective function is of the form +`\begin{equation} + f(\wv) := + \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) + + \lambda\, R(\wv_i) + \label{eq:regPrimal} + \ . +\end{equation}` +Here the vectors `$\x_i\in\R^d$` are the training data examples, for `$1\le i\le n$`, and +`$y_i\in\R$` are their corresponding labels, which we want to predict. +We call the method *linear* if $L(\wv; \x, y)$ can be expressed as a function of $\wv^T x$ and $y$. +Several MLlib's classification and regression algorithms fall into this category, +and are discussed here. + +The objective function `$f$` has two parts: +the loss that measures the error of the model on the training data, +and the regularizer that measures the complexity of the model. +The loss function `$L(\wv;.)$` must be a convex function in `$\wv$`. +The fixed regularization parameter `$\lambda \ge 0$` (`regParam` in the code) defines the trade-off +between the two goals of small loss and small model complexity. + +### Loss functions + +The following table summarizes the loss functions and their gradients or sub-gradients for the +methods MLlib supports: + + + + + + + + + + + + + + + + + + + +
loss function $L(\wv; \x, y)$gradient or sub-gradient
hinge loss$\max \{0, 1-y \wv^T \x \}, \quad y \in \{-1, +1\}$$\begin{cases}-y \cdot \x & \text{if $y \wv^T \x <1$}, \\ 0 & +\text{otherwise}.\end{cases}$
logistic loss$\log(1+\exp( -y \wv^T \x)), \quad y \in \{-1, +1\}$$-y \left(1-\frac1{1+\exp(-y \wv^T \x)} \right) \cdot \x$
squared loss$\frac{1}{2} (\wv^T \x - y)^2, \quad y \in \R$$(\wv^T \x - y) \cdot \x$
+ +### Regularizers + +The purpose of the [regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to +encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid +over-fitting. +We support the following regularizers in MLlib: + + + + + + + + + + + + + + + + +
regularizer $R(\wv)$gradient or sub-gradient
zero (unregularized)0$\0$
L2$\frac{1}{2}\|\wv\|_2^2$$\wv$
L1$\|\wv\|_1$$\mathrm{sign}(\wv)$
+ +Here `$\mathrm{sign}(\wv)$` is the vector consisting of the signs (`$\pm1$`) of all the entries +of `$\wv$`. + +L2-regularized problems are generally easier to solve than L1-regularized due to smoothness. +However, L1 regularization can help promote sparsity in weights, leading to simpler models, which is +also used for feature selection. It is not recommended to train models without any regularization, +especially when the number of training examples is small. + +## Binary classification + +[Binary classification](http://en.wikipedia.org/wiki/Binary_classification) is to divide items into +two categories: positive and negative. MLlib supports two linear methods for binary classification: +linear support vector machine (SVM) and logistic regression. The training data set is represented +by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the mathematical +formulation, a training label $y$ is either $+1$ (positive) or $-1$ (negative), which is convenient +for the formulation. *However*, the negative label is represented by $0$ in MLlib instead of $-1$, +to be consistent with multiclass labeling. + +### Linear support vector machine (SVM) + +The [linear SVM](http://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM) +has become a standard choice for large-scale classification tasks. +The name "linear SVM" is actually ambiguous. +By "linear SVM", we mean specifically the linear method with the loss function in formulation +`$\eqref{eq:regPrimal}$` given by the hinge loss +`\[ +L(\wv;\x,y) := \max \{0, 1-y \wv^T \x \}. +\]` +By default, linear SVMs are trained with an L2 regularization. +We also support alternative L1 regularization. In this case, +the problem becomes a [linear program](http://en.wikipedia.org/wiki/Linear_programming). + +Linear SVM algorithm outputs a SVM model, which makes predictions based on the value of $\wv^T \x$. +By the default, if $\wv^T \x \geq 0$, the outcome is positive, or negative otherwise. +However, quite often in practice, the default threshold $0$ is not a good choice. +The threshold should be determined via model evaluation. + +### Logistic regression + +[Logistic regression](http://en.wikipedia.org/wiki/Logistic_regression) is widely used to predict a +binary response. It is a linear method with the loss function in formulation +`$\eqref{eq:regPrimal}$` given by the logistic loss +`\[ +L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)). +\]` + +Logistic regression algorithm outputs a logistic regression model, which makes predictions by +applying the logistic function +`\[ +\mathrm{logit}(z) = \frac{1}{1 + e^{-z}} +\]` +$\wv^T \x$. +By default, if $\mathrm{logit}(\wv^T x) > 0.5$, the outcome is positive, or negative otherwise. +For the same reason mentioned above, quite often in practice, this default threshold is not a good choice. +The threshold should be determined via model evaluation. + +### Evaluation metrics + +MLlib supports common evaluation metrics for binary classification (not available in Python). This +includes precision, recall, [F-measure](http://en.wikipedia.org/wiki/F1_score), +[receiver operating characteristic (ROC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic), +precision-recall curve, and +[area under the curves (AUC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). +Among the metrics, area under ROC is commonly used to compare models and precision/recall/F-measure +can help determine the threshold to use. + +### Examples + +
+ +
+The following code snippet illustrates how to load a sample dataset, execute a +training algorithm on this training data using a static method in the algorithm +object, and make predictions with the resulting model to compute the training +error. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.classification.SVMWithSGD +import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +// Load training data in LIBSVM format. +val data = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") + +// Split data into training (60%) and test (40%). +val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) +val training = splits(0).cache() +val test = splits(1) + +// Run training algorithm to build the model +val numIterations = 100 +val model = SVMWithSGD.train(training, numIterations) + +// Clear the default threshold. +model.clearThreshold() + +// Compute raw scores on the test set. +val scoreAndLabels = test.map { point => + val score = model.predict(point.features) + (score, point.label) +} + +// Get evaluation metrics. +val metrics = new BinaryClassificationMetrics(scoreAndLabels) +val auROC = metrics.areaUnderROC() + +println("Area under ROC = " + auROC) +{% endhighlight %} + +The `SVMWithSGD.train()` method by default performs L2 regularization with the +regularization parameter set to 1.0. If we want to configure this algorithm, we +can customize `SVMWithSGD` further by creating a new object directly and +calling setter methods. All other MLlib algorithms support customization in +this way as well. For example, the following code produces an L1 regularized +variant of SVMs with regularization parameter set to 0.1, and runs the training +algorithm for 200 iterations. + +{% highlight scala %} +import org.apache.spark.mllib.optimization.L1Updater + +val svmAlg = new SVMWithSGD() +svmAlg.optimizer.setNumIterations(200) + .setRegParam(0.1) + .setUpdater(new L1Updater) +val modelL1 = svmAlg.run(parsedData) +{% endhighlight %} + +Similarly, you can use replace `SVMWithSGD` by +[`LogisticRegressionWithSGD`](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). + +
+ +
+All of MLlib's methods use Java-friendly types, so you can import and call them there the same +way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the +Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by +calling `.rdd()` on your `JavaRDD` object. +
+ +
+The following example shows how to load a sample dataset, build Logistic Regression model, +and make predictions with the resulting model to compute the training error. + +{% highlight python %} +from pyspark.mllib.classification import LogisticRegressionWithSGD +from pyspark.mllib.regression import LabeledPoint +from numpy import array + +# Load and parse the data +def parsePoint(line): + values = [float(x) for x in line.split(' ')] + return LabeledPoint(values[0], values[1:]) + +data = sc.textFile("mllib/data/sample_svm_data.txt") +parsedData = data.map(parsePoint) + +# Build the model +model = LogisticRegressionWithSGD.train(parsedData) + +# Evaluating the model on training data +labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) +trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) +print("Training Error = " + str(trainErr)) +{% endhighlight %} +
+
+ +## Linear least squares, Lasso, and ridge regression + +Linear least squares is a family of linear methods with the loss function in formulation +`$\eqref{eq:regPrimal}$` given by the squared loss + +`\[ +L(\wv;\x,y) := \frac{1}{2} (\wv^T \x - y)^2. +\]` + +Depending on the regularization type, we call the method +[*ordinary least squares*](http://en.wikipedia.org/wiki/Ordinary_least_squares) or simply +[*linear least squares*](http://en.wikipedia.org/wiki/Linear_least_squares_(mathematics)) if there +is no regularization, [*ridge regression*](http://en.wikipedia.org/wiki/Ridge_regression) if L2 +regularization is used, and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) if L1 +regularization is used. This average loss $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$ is also +known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). + +Note that the squared loss is sensitive to outliers. +Regularization or a robust alternative (e.g., $\ell_1$ regression) is usually necessary in practice. + +### Examples + +
+ +
+The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. +The example then uses LinearRegressionWithSGD to build a simple linear model to predict label +values. We compute the Mean Squared Error at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +{% highlight scala %} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors + +// Load and parse the data +val data = sc.textFile("mllib/data/ridge-data/lpsa.data") +val parsedData = data.map { line => + val parts = line.split(',') + LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) +} + +// Building the model +val numIterations = 100 +val model = LinearRegressionWithSGD.train(parsedData, numIterations) + +// Evaluate model on training examples and compute training error +val valuesAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.reduce(_ + _) / valuesAndPreds.count +println("training Mean Squared Error = " + MSE) +{% endhighlight %} + +Similarly you can use +[`RidgeRegressionWithSGD`](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +and [`LassoWithSGD`](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD). + +
+ +
+All of MLlib's methods use Java-friendly types, so you can import and call them there the same +way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the +Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by +calling `.rdd()` on your `JavaRDD` object. +
+ +
+The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. +The example then uses LinearRegressionWithSGD to build a simple linear model to predict label +values. We compute the Mean Squared Error at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD +from numpy import array + +# Load and parse the data +def parsePoint(line): + values = [float(x) for x in line.replace(',', ' ').split(' ')] + return LabeledPoint(values[0], values[1:]) + +data = sc.textFile("mllib/data/ridge-data/lpsa.data") +parsedData = data.map(parsePoint) + +# Build the model +model = LinearRegressionWithSGD.train(parsedData) + +# Evaluate the model on training data +valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) +MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y) / valuesAndPreds.count() +print("Mean Squared Error = " + str(MSE)) +{% endhighlight %} +
+
+ +## Implementation (developer) + +Behind the scene, MLlib implements a simple distributed version of stochastic gradient descent +(SGD), building on the underlying gradient descent primitive (as described in the optimization section). All provided algorithms take as input a +regularization parameter (`regParam`) along with various parameters associated with stochastic +gradient descent (`stepSize`, `numIterations`, `miniBatchFraction`). For each of them, we support +all three possible regularizations (none, L1 or L2). + +Algorithms are all implemented in Scala: + +* [SVMWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.SVMWithSGD) +* [LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) +* [LinearRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) +* [RidgeRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +* [LassoWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD) + +Python calls the Scala implementation via +[PythonMLLibAPI](api/mllib/index.html#org.apache.spark.mllib.api.python.PythonMLLibAPI). diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md new file mode 100644 index 0000000000000..6160fe5b2fe8c --- /dev/null +++ b/docs/mllib-naive-bayes.md @@ -0,0 +1,115 @@ +--- +layout: global +title: MLlib - Naive Bayes +--- + +Naive Bayes is a simple multiclass classification algorithm with the assumption of independence +between every pair of features. Naive Bayes can be trained very efficiently. Within a single pass to +the training data, it computes the conditional probability distribution of each feature given label, +and then it applies Bayes' theorem to compute the conditional probability distribution of label +given an observation and use it for prediction. For more details, please visit the wikipedia page +[Naive Bayes classifier](http://en.wikipedia.org/wiki/Naive_Bayes_classifier). + +In MLlib, we implemented multinomial naive Bayes, which is typically used for document +classification. Within that context, each observation is a document, each feature represents a term, +whose value is the frequency of the term. For its formulation, please visit the wikipedia page +[Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) +or the section +[Naive Bayes text classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) +from the book Introduction to Information +Retrieval. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by +setting the parameter $\lambda$ (default to $1.0$). For document classification, the input feature +vectors are usually sparse. Please supply sparse vectors as input to take advantage of +sparsity. Since the training data is only used once, it is not necessary to cache it. + +## Examples + +
+
+ +[NaiveBayes](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements +multinomial naive Bayes. It takes an RDD of +[LabeledPoint](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an optional +smoothing parameter `lambda` as input, and output a +[NaiveBayesModel](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which +can be used for evaluation and prediction. + +{% highlight scala %} +import org.apache.spark.mllib.classification.NaiveBayes + +val training: RDD[LabeledPoint] = ... // training set +val test: RDD[LabeledPoint] = ... // test set + +val model = NaiveBayes.train(training, lambda = 1.0) +val prediction = model.predict(test.map(_.features)) + +val predictionAndLabel = prediction.zip(test.map(_.label)) +val accuracy = 1.0 * predictionAndLabel.filter(x => x._1 == x._2).count() / test.count() +{% endhighlight %} +
+ +
+ +[NaiveBayes](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements +multinomial naive Bayes. It takes a Scala RDD of +[LabeledPoint](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an +optionally smoothing parameter `lambda` as input, and output a +[NaiveBayesModel](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which +can be used for evaluation and prediction. + +{% highlight java %} +import org.apache.spark.mllib.classification.NaiveBayes; + +JavaRDD training = ... // training set +JavaRDD test = ... // test set + +NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); + +JavaRDD prediction = model.predict(test.map(new Function() { + public Vector call(LabeledPoint p) { + return p.features(); + } + }) +JavaPairRDD predictionAndLabel = + prediction.zip(test.map(new Function() { + public Double call(LabeledPoint p) { + return p.label(); + } + }) +double accuracy = 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + public Boolean call(Tuple2 pl) { + return pl._1() == pl._2(); + } + }).count() / test.count() +{% endhighlight %} +
+ +
+ +[NaiveBayes](api/pyspark/pyspark.mllib.classification.NaiveBayes-class.html) implements multinomial +naive Bayes. It takes an RDD of +[LabeledPoint](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html) and an optionally +smoothing parameter `lambda` as input, and output a +[NaiveBayesModel](api/pyspark/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be +used for evaluation and prediction. + + +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.classification import NaiveBayes + +# an RDD of LabeledPoint +data = sc.parallelize([ + LabeledPoint(0.0, [0.0, 0.0]) + ... # more labeled points +]) + +# Train a naive Bayes model. +model = NaiveBayes.train(data, 1.0) + +# Make prediction. +prediction = model.predict([0.0, 0.0]) +{% endhighlight %} + +
+
diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 396b98d52a994..bec3912b55dc7 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -1,6 +1,6 @@ --- layout: global -title: MLlib - Optimization +title: MLlib - Optimization --- * Table of contents @@ -25,9 +25,10 @@ title: MLlib - Optimization -# Mathematical Description +## Mathematical description + +### Gradient descent -## (Sub)Gradient Descent The simplest method to solve optimization problems of the form `$\min_{\wv \in\R^d} \; f(\wv)$` is [gradient descent](http://en.wikipedia.org/wiki/Gradient_descent). Such first-order optimization methods (including gradient descent and stochastic variants @@ -38,14 +39,14 @@ the direction of steepest descent, which is the negative of the derivative (call [gradient](http://en.wikipedia.org/wiki/Gradient)) of the function at the current point, i.e., at the current parameter value. If the objective function `$f$` is not differentiable at all arguments, but still convex, then a -*subgradient* +*sub-gradient* is the natural generalization of the gradient, and assumes the role of the step direction. -In any case, computing a gradient or subgradient of `$f$` is expensive --- it requires a full +In any case, computing a gradient or sub-gradient of `$f$` is expensive --- it requires a full pass through the complete dataset, in order to compute the contributions from all loss terms. -## Stochastic (Sub)Gradient Descent (SGD) +### Stochastic gradient descent (SGD) Optimization problems whose objective function `$f$` is written as a sum are particularly -suitable to be solved using *stochastic subgradient descent (SGD)*. +suitable to be solved using *stochastic gradient descent (SGD)*. In our case, for the optimization formulations commonly used in supervised machine learning, `\begin{equation} @@ -95,12 +96,12 @@ As an alternative to just use the subgradient `$R'(\wv)$` of the regularizer in direction, an improved update for some cases can be obtained by using the proximal operator instead. For the L1-regularizer, the proximal operator is given by soft thresholding, as implemented in -[L1Updater](api/mllib/index.html#org.apache.spark.mllib.optimization.L1Updater). +[L1Updater](api/scala/index.html#org.apache.spark.mllib.optimization.L1Updater). -## Update Schemes for Distributed SGD +### Update schemes for distributed SGD The SGD implementation in -[GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) uses +[GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) uses a simple (distributed) sampling of the data examples. We recall that the loss part of the optimization problem `$\eqref{eq:regPrimal}$` is `$\frac1n \sum_{i=1}^n L(\wv;\x_i,y_i)$`, and therefore `$\frac1n \sum_{i=1}^n L'_{\wv,i}$` would @@ -129,16 +130,16 @@ point. -# Implementation in MLlib +## Implementation in MLlib Gradient descent methods including stochastic subgradient descent (SGD) as included as a low-level primitive in `MLlib`, upon which various ML algorithms are developed, see the -classification and regression +linear methods section for example. The SGD method -[GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) +[GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) has the following parameters: * `gradient` is a class that computes the stochastic gradient of the function @@ -162,5 +163,3 @@ each iteration, to compute the gradient direction. Available algorithms for gradient descent: * [GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) - - diff --git a/docs/monitoring.md b/docs/monitoring.md index 4c91c3a5929bf..6f35fc37c4075 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -39,22 +39,33 @@ You can start a the history server by executing: The base logging directory must be supplied, and should contain sub-directories that each represents an application's event logs. This creates a web interface at -`http://:18080` by default. The history server depends on the following variables: +`http://:18080` by default. The history server can be configured as follows: - + + + + + + + + +
Environment VariableMeaning
SPARK_DAEMON_MEMORYMemory to allocate to the history server. (default: 512m).Memory to allocate to the history server (default: 512m).
SPARK_DAEMON_JAVA_OPTS JVM options for the history server (default: none).
SPARK_PUBLIC_DNS + The public address for the history server. If this is not set, links to application history + may use the internal address of the server, resulting in broken links (default: none). +
SPARK_HISTORY_OPTS + spark.history.* configuration options for the history server (default: none). +
-Further, the history server can be configured as follows: - @@ -80,6 +91,43 @@ Further, the history server can be configured as follows: The port to which the web interface of the history server binds. + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.history.kerberos.enabledfalse + Indicates whether the history server should use kerberos to login. This is useful + if the history server is accessing HDFS files on a secure Hadoop cluster. If this is + true it looks uses the configs spark.history.kerberos.principal and + spark.history.kerberos.keytab. +
spark.history.kerberos.principal(none) + Kerberos principal name for the History Server. +
spark.history.kerberos.keytab(none) + Location of the kerberos keytab file for the History Server. +
spark.history.ui.acls.enablefalse + Specifies whether acls should be checked to authorize users viewing the applications. + If enabled, access control checks are made regardless of what the individual application had + set for spark.ui.acls.enable when the application was run. The application owner + will always have authorization to view their own application and any users specified via + spark.ui.view.acls when the application was run will also have authorization + to view that application. + If disabled, no access control checks are made. +
Note that in all of these UIs, the tables are sortable by clicking their headers, diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 39de603b29f87..98233bf556b79 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -134,7 +134,7 @@ Files listed here will be added to the `PYTHONPATH` and shipped to remote worker Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. You can set [configuration properties](configuration.html#spark-properties) by passing a -[SparkConf](api/pyspark/pyspark.conf.SparkConf-class.html) object to SparkContext: +[SparkConf](api/python/pyspark.conf.SparkConf-class.html) object to SparkContext: {% highlight python %} from pyspark import SparkConf, SparkContext @@ -147,7 +147,7 @@ sc = SparkContext(conf = conf) # API Docs -[API documentation](api/pyspark/index.html) for PySpark is available as Epydoc. +[API documentation](api/python/index.html) for PySpark is available as Epydoc. Many of the methods also contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples. # Libraries diff --git a/docs/quick-start.md b/docs/quick-start.md index 60e8b1ba0eb46..64996b52e0404 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -99,23 +99,32 @@ scala> linesWithSpark.count() res9: Long = 15 {% endhighlight %} -It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). +It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is +that these same functions can be used on very large data sets, even when they are striped across +tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to +a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). -# A Standalone App in Scala -Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. +# A Standalone Application +Now say we wanted to write a standalone application using the Spark API. We will walk through a +simple application in both Scala (with SBT), Java (with Maven), and Python. -We'll create a very simple Spark application in Scala. So simple, in fact, that it's named `SimpleApp.scala`: +
+
+ +We'll create a very simple Spark application in Scala. So simple, in fact, that it's +named `SimpleApp.scala`: {% highlight scala %} /*** SimpleApp.scala ***/ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.SparkConf object SimpleApp { def main(args: Array[String]) { - val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system - val sc = new SparkContext("local", "Simple App", "YOUR_SPARK_HOME", - List("target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar")) + val logFile = "YOUR_SPARK_HOME/README.md" // Should be some file on your system + val conf = new SparkConf().setAppName("Simple Application") + val sc = new SparkContext(conf) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() @@ -124,9 +133,19 @@ object SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. +This program just counts the number of lines containing 'a' and the number containing 'b' in the +Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is +installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, +we initialize a SparkContext as part of the program. + +We pass the SparkContext constructor a +[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) +object which contains information about our +application. We also call sc.addJar to make sure that when our application is launched in cluster +mode, the jar file containing it will be shipped automatically to worker nodes. -This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on: +This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` +which explains that Spark is a dependency. This file also adds a repository that Spark depends on: {% highlight scala %} name := "Simple Project" @@ -140,15 +159,12 @@ libraryDependencies += "org.apache.spark" %% "spark-core" % "{{site.SPARK_VERSIO resolvers += "Akka Repository" at "http://repo.akka.io/releases/" {% endhighlight %} -If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS: - -{% highlight scala %} -libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "" -{% endhighlight %} - -Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt/sbt run` to execute our program. +For sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` +according to the typical directory structure. Once that is in place, we can create a JAR package +containing the application's code, then use the `spark-submit` script to run our program. {% highlight bash %} +# Your directory layout should look like this $ find . . ./simple.sbt @@ -157,27 +173,37 @@ $ find . ./src/main/scala ./src/main/scala/SimpleApp.scala -$ sbt/sbt package -$ sbt/sbt run +# Package a jar containing your application +$ sbt package +... +[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar + +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit \ + --class "SimpleApp" \ + --master local[4] \ + target/scala-2.10/simple-project_2.10-1.0.jar ... Lines with a: 46, Lines with b: 23 {% endhighlight %} -# A Standalone App in Java -Now say we wanted to write a standalone application using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. +
+
+This example will use Maven to compile an application jar, but any similar build system will work. We'll create a very simple Spark application, `SimpleApp.java`: {% highlight java %} /*** SimpleApp.java ***/ import org.apache.spark.api.java.*; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.Function; public class SimpleApp { public static void main(String[] args) { - String logFile = "$YOUR_SPARK_HOME/README.md"; // Should be some file on your system - JavaSparkContext sc = new JavaSparkContext("local", "Simple App", - "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"}); + String logFile = "YOUR_SPARK_HOME/README.md"; // Should be some file on your system + SparkConf conf = new SparkConf().setAppName("Simple Application"); + JavaSparkContext sc = new JavaSparkContext(conf); JavaRDD logData = sc.textFile(logFile).cache(); long numAs = logData.filter(new Function() { @@ -193,9 +219,16 @@ public class SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail. +This program just counts the number of lines containing 'a' and the number containing 'b' in a text +file. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. +As with the Scala example, we initialize a SparkContext, though we use the special +`JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by +`JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes +that extend `spark.api.java.function.Function`. The +[Java programming guide](java-programming-guide.html) describes these differences in more detail. -To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version. +To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. +Note that Spark artifacts are tagged with a Scala version. {% highlight xml %} @@ -221,16 +254,6 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a {% endhighlight %} -If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS: - -{% highlight xml %} - - org.apache.hadoop - hadoop-client - ... - -{% endhighlight %} - We lay out these files according to the canonical Maven directory structure: {% highlight bash %} $ find . @@ -241,16 +264,26 @@ $ find . ./src/main/java/SimpleApp.java {% endhighlight %} -Now, we can execute the application using Maven: +Now, we can package the application using Maven and execute it with `./bin/spark-submit`. {% highlight bash %} +# Package a jar containing your application $ mvn package -$ mvn exec:java -Dexec.mainClass="SimpleApp" +... +[INFO] Building jar: {..}/{..}/target/simple-project-1.0.jar + +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit \ + --class "SimpleApp" \ + --master local[4] \ + target/simple-project-1.0.jar ... Lines with a: 46, Lines with b: 23 {% endhighlight %} -# A Standalone App in Python +
+
+ Now we will show how to write a standalone application using the Python API (PySpark). As an example, we'll create a simple Spark application, `SimpleApp.py`: @@ -259,7 +292,7 @@ As an example, we'll create a simple Spark application, `SimpleApp.py`: """SimpleApp.py""" from pyspark import SparkContext -logFile = "$YOUR_SPARK_HOME/README.md" # Should be some file on your system +logFile = "YOUR_SPARK_HOME/README.md" # Should be some file on your system sc = SparkContext("local", "Simple App") logData = sc.textFile(logFile).cache() @@ -270,11 +303,15 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs) {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. -Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. +This program just counts the number of lines containing 'a' and the number containing 'b' in a +text file. +Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala and Java examples, we use a SparkContext to create RDDs. -We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference. -For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html). +We can pass Python functions to Spark, which are automatically serialized along with any variables +that they reference. +For applications that use custom classes or third-party libraries, we can add those code +dependencies to SparkContext to ensure that they will be available on remote machines; this is +described in more detail in the [Python programming guide](python-programming-guide.html). `SimpleApp` is simple enough that we do not need to specify any code dependencies. We can run this application using the `bin/pyspark` script: @@ -286,57 +323,12 @@ $ ./bin/pyspark SimpleApp.py Lines with a: 46, Lines with b: 23 {% endhighlight python %} -# Running on a Cluster - -There are a few additional considerations when running applicaitons on a -[Spark](spark-standalone.html), [YARN](running-on-yarn.html), or -[Mesos](running-on-mesos.html) cluster. - -### Including Your Dependencies -If your code depends on other projects, you will need to ensure they are also -present on the slave nodes. A popular approach is to 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-assembly-plugin/) -have assembly plugins. When creating assembly jars, list Spark -itself as a `provided` dependency; it need not be bundled since it is -already present on the slaves. Once you have an assembled jar, -add it to the SparkContext as shown here. It is also possible to add -your dependent jars one-by-one using the `addJar` method of `SparkContext`. - -For Python, you can use the `pyFiles` argument of SparkContext -or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. - -### Setting Configuration Options -Spark includes several [configuration options](configuration.html#spark-properties) -that influence the behavior of your application. -These should be set by building a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) -object and passing it to the SparkContext constructor. -For example, in Java and Scala, you can do: - -{% highlight scala %} -import org.apache.spark.{SparkConf, SparkContext} -val conf = new SparkConf() - .setMaster("local") - .setAppName("My application") - .set("spark.executor.memory", "1g") -val sc = new SparkContext(conf) -{% endhighlight %} - -Or in Python: - -{% highlight scala %} -from pyspark import SparkConf, SparkContext -conf = SparkConf() -conf.setMaster("local") -conf.setAppName("My application") -conf.set("spark.executor.memory", "1g")) -sc = SparkContext(conf = conf) -{% endhighlight %} +
+
-### Accessing Hadoop Filesystems +# Where to go from here +Congratulations on running your first Spark application! -The examples here access a local file. To read data from a distributed -filesystem, such as HDFS, include -[Hadoop version information](index.html#a-note-about-hadoop-versions) -in your build file. By default, Spark builds against HDFS 1.0.4. +* For an in-depth overview of the API see "Programming Guides" menu section. +* For running applications on a cluster head to the [deployment overview](cluster-overview.html). +* For configuration options available to Spark applications see the [configuration page](configuration.html). diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 2b0a51e9dfc54..b8d89cf00ffbd 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -25,45 +25,53 @@ To write a Spark application, you need to add a dependency on Spark. If you use artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION}} -In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS: +In addition, if you wish to access an HDFS cluster, you need to add a dependency on +`hadoop-client` for your version of HDFS. Some common HDFS version tags are listed on the +[third party distributions](hadoop-third-party-distributions.html) page. groupId = org.apache.hadoop artifactId = hadoop-client version = -For other build systems, you can run `sbt/sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). - Finally, you need to import some Spark classes and implicit conversions into your program. Add the following lines: {% highlight scala %} import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.SparkConf {% endhighlight %} # Initializing Spark -The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark how to access a cluster. -This is done through the following constructor: +The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark +how to access a cluster. To create a `SparkContext` you first need to build a `SparkConf` object +that contains information about your application. {% highlight scala %} -new SparkContext(master, appName, [sparkHome], [jars]) +val conf = new SparkConf().setAppName().setMaster() +new SparkContext(conf) {% endhighlight %} -or through `new SparkContext(conf)`, which takes a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) -object for more advanced configuration. - -The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. +The `` parameter is a string specifying a [Spark, Mesos or YARN cluster URL](#master-urls) +to connect to, or a special "local" string to run in local mode, as described below. `` is +a name for your application, which will be shown in the cluster web UI. It's also possible to set +these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file) +which avoids hard-coding the master name in your application. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the +variable called `sc`. Making your own SparkContext will not work. You can set which master the +context connects to using the `--master` argument, and you can add JARs to the classpath +by passing a comma separated list to the `--jars` argument. For example, to run +`bin/spark-shell` on exactly four cores, use {% highlight bash %} -$ MASTER=local[4] ./bin/spark-shell +$ ./bin/spark-shell --master local[4] {% endhighlight %} Or, to also add `code.jar` to its classpath, use: {% highlight bash %} -$ MASTER=local[4] ADD_JARS=code.jar ./bin/spark-shell +$ ./bin/spark-shell --master local[4] --jars code.jar {% endhighlight %} ### Master URLs @@ -83,21 +91,16 @@ The master URL passed to Spark can be in one of the following formats: The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, which is 5050 by default. + yarn-client Connect to a YARN cluster in +client mode. The cluster location will be inferred based on the local Hadoop configuration. + + yarn-cluster Connect to a YARN cluster in +cluster mode. The cluster location will be inferred based on the local Hadoop configuration. + If no master URL is specified, the spark shell defaults to "local[*]". -For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details. - -### Deploying Code on a Cluster - -If you want to run your application on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code: - -* `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them). -* `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies. - -If you run `bin/spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./bin/spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed. - # Resilient Distributed Datasets (RDDs) Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are currently two types of RDDs: *parallelized collections*, which take an existing Scala collection and run functions on it in parallel, and *Hadoop datasets*, which run functions on each record of a file in Hadoop distributed file system or any other storage system supported by Hadoop. Both types of RDDs can be operated on through the same methods. @@ -145,7 +148,7 @@ All transformations in Spark are lazy, in that they do not compute their By default, each transformed RDD is recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options. -The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD) for details): +The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD) for details): ### Transformations @@ -214,7 +217,7 @@ The following tables list the transformations and actions currently supported (s -A complete list of transformations is available in the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD). +A complete list of transformations is available in the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD). ### Actions @@ -262,7 +265,7 @@ A complete list of transformations is available in the [RDD API doc](api/core/in -A complete list of actions is available in the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD). +A complete list of actions is available in the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD). ## RDD Persistence @@ -281,7 +284,7 @@ In addition, each RDD can be stored using a different *storage level*, allowing persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or replicate it across nodes, or store the data in off-heap memory in [Tachyon](http://tachyon-project.org/). These levels are chosen by passing a -[`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) +[`org.apache.spark.storage.StorageLevel`](api/scala/index.html#org.apache.spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is: @@ -353,7 +356,7 @@ waiting to recompute a lost partition. If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the -[`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. +[`StorageLevel`](api/scala/index.html#org.apache.spark.storage.StorageLevel$) singleton object. Spark has a block manager inside the Executors that let you chose memory, disk, or off-heap. The latter is for storing RDDs off-heap outside the Executor JVM on top of the memory management system diff --git a/docs/security.md b/docs/security.md index 9e4218fbcfe7d..90c69915f517f 100644 --- a/docs/security.md +++ b/docs/security.md @@ -7,6 +7,8 @@ Spark currently supports authentication via a shared secret. Authentication can The Spark UI can also be secured by using javax servlet filters. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view acls to make sure they are authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' control the behavior of the acls. Note that the person who started the application always has view access to the UI. +If your applications are using event logging, the directory where the event logs go (`spark.eventLog.dir`) should be manually created and have the proper permissions set on it. If you want those log files secure, the permissions should be set to drwxrwxrwxt for that directory. The owner of the directory should be the super user who is running the history server and the group permissions should be restricted to super user group. This will allow all users to write to the directory but will prevent unprivileged users from removing or renaming a file unless they own the file or directory. The event log files will be created by Spark with permissions such that only the user and group have read and write access. + For Spark on Yarn deployments, configuring `spark.authenticate` to true will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. The Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI. For other types of Spark deployments, the spark config `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. The UI can be secured using a javax servlet filter installed via `spark.ui.filters`. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI. diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md index 891c2bfa8943d..35d06c51aaf0e 100644 --- a/docs/spark-debugger.md +++ b/docs/spark-debugger.md @@ -39,7 +39,7 @@ where `path/to/event-log` is where you want the event log to go relative to `$SP ### Loading the event log into the debugger -1. Run a Spark shell with `MASTER=host ./bin/spark-shell`. +1. Run a Spark shell with `./bin/spark-shell --master hist`. 2. Use `EventLogReader` to load the event log as follows: {% highlight scala %} spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log")) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 7e4eea323aa63..dc7f206e03996 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -139,12 +139,12 @@ constructor](scala-programming-guide.html#initializing-spark). To run an interactive Spark shell against the cluster, run the following command: - MASTER=spark://IP:PORT ./bin/spark-shell + ./bin/spark-shell --master spark://IP:PORT Note that if you are running spark-shell from one of the spark cluster machines, the `bin/spark-shell` script will automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`. -You can also pass an option `-c ` to control the number of cores that spark-shell uses on the cluster. +You can also pass an option `--cores ` to control the number of cores that spark-shell uses on the cluster. # Launching Compiled Spark Applications diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 6f616fb7c2448..0c743c9d6010d 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -14,8 +14,8 @@ title: Spark SQL Programming Guide Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed -[Row](api/sql/catalyst/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with +[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed +[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). @@ -27,8 +27,8 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac
Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[JavaSchemaRDD](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed -[Row](api/sql/catalyst/index.html#org.apache.spark.sql.api.java.Row) objects along with +[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed +[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects along with a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). @@ -38,8 +38,8 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/pyspark/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed -[Row](api/pyspark/pyspark.sql.Row-class.html) objects along with +[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed +[Row](api/python/pyspark.sql.Row-class.html) objects along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). @@ -56,7 +56,7 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac
The entry point into all relational functionality in Spark is the -[SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its +[SQLContext](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its descendants. To create a basic SQLContext, all you need is a SparkContext. {% highlight scala %} @@ -69,10 +69,10 @@ import sqlContext._
-
+
The entry point into all relational functionality in Spark is the -[JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one +[JavaSQLContext](api/scala/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one of its descendants. To create a basic JavaSQLContext, all you need is a JavaSparkContext. {% highlight java %} @@ -85,7 +85,7 @@ JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx);
The entry point into all relational functionality in Spark is the -[SQLContext](api/pyspark/pyspark.sql.SQLContext-class.html) class, or one +[SQLContext](api/python/pyspark.sql.SQLContext-class.html) class, or one of its decedents. To create a basic SQLContext, all you need is a SparkContext. {% highlight python %} @@ -114,6 +114,8 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext._ // Define the schema using a case class. +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) // Create an RDD of Person objects and register it as a table. @@ -143,19 +145,19 @@ public static class Person implements Serializable { private String name; private int age; - String getName() { + public String getName() { return name; } - void setName(String name) { + public void setName(String name) { this.name = name; } - int getAge() { + public int getAge() { return age; } - void setAge(int age) { + public void setAge(int age) { this.age = age; } } @@ -331,7 +333,7 @@ val teenagers = people.where('age >= 10).where('age <= 19).select('name) The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are evaluated by the SQL execution engine. A full list of the functions supported can be found in the -[ScalaDoc](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). +[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 3fb540c9fb560..3cfa4516cc12d 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -9,7 +9,7 @@ This guide shows the programming model and features by walking through a simple ### Writing a Simple Receiver -This starts with implementing [NetworkReceiver](api/streaming/index.html#org.apache.spark.streaming.dstream.NetworkReceiver). +This starts with implementing [NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver). The following is a simple socket text-stream receiver. @@ -125,4 +125,4 @@ _A more comprehensive example is provided in the spark streaming examples_ ## References 1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) -2.[NetworkReceiver](api/streaming/index.html#org.apache.spark.streaming.dstream.NetworkReceiver) +2.[NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index f9904d45013f6..7ad06427cac5c 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -40,7 +40,7 @@ Spark Streaming provides a high-level abstraction called *discretized stream* or which represents a continuous stream of data. DStreams can be created either from input data stream from sources such as Kafka and Flume, or by applying high-level operations on other DStreams. Internally, a DStream is represented as a sequence of -[RDDs](api/core/index.html#org.apache.spark.rdd.RDD). +[RDDs](api/scala/index.html#org.apache.spark.rdd.RDD). This guide shows you how to start writing Spark Streaming programs with DStreams. You can write Spark Streaming programs in Scala or Java, both of which are presented in this guide. You @@ -62,7 +62,7 @@ First, we import the names of the Spark Streaming classes, and some implicit conversions from StreamingContext into our environment, to add useful methods to other classes we need (like DStream). -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) is the +[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) is the main entry point for all streaming functionality. {% highlight scala %} @@ -71,7 +71,7 @@ import org.apache.spark.streaming.StreamingContext._ {% endhighlight %} Then we create a -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) object. +[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object. Besides Spark's configuration, we specify that any DStream will be processed in 1 second batches. @@ -132,7 +132,7 @@ The complete code can be found in the Spark Streaming example
First, we create a -[JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object, +[JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object, which is the main entry point for all streaming functionality. Besides Spark's configuration, we specify that any DStream would be processed in 1 second batches. @@ -168,7 +168,7 @@ JavaDStream words = lines.flatMap( generating multiple new records from each record in the source DStream. In this case, each line will be split into multiple words and the stream of words is represented as the `words` DStream. Note that we defined the transformation using a -[FlatMapFunction](api/core/index.html#org.apache.spark.api.java.function.FlatMapFunction) object. +[FlatMapFunction](api/scala/index.html#org.apache.spark.api.java.function.FlatMapFunction) object. As we will discover along the way, there are a number of such convenience classes in the Java API that help define DStream transformations. @@ -192,9 +192,9 @@ wordCounts.print(); // Print a few of the counts to the console {% endhighlight %} The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word, -1)` pairs, using a [PairFunction](api/core/index.html#org.apache.spark.api.java.function.PairFunction) +1)` pairs, using a [PairFunction](api/scala/index.html#org.apache.spark.api.java.function.PairFunction) object. Then, it is reduced to get the frequency of words in each batch of data, -using a [Function2](api/core/index.html#org.apache.spark.api.java.function.Function2) object. +using a [Function2](api/scala/index.html#org.apache.spark.api.java.function.Function2) object. Finally, `wordCounts.print()` will print a few of the counts generated every second. Note that when these lines are executed, Spark Streaming only sets up the computation it @@ -272,12 +272,10 @@ Time: 1357008430000 ms -If you plan to run the Scala code for Spark Streaming-based use cases in the Spark -shell, you should start the shell with the SparkConfiguration pre-configured to -discard old batches periodically: +You can also use Spark Streaming directly from the Spark shell: {% highlight bash %} -$ SPARK_JAVA_OPTS=-Dspark.cleaner.ttl=10000 bin/spark-shell +$ bin/spark-shell {% endhighlight %} ... and create your StreamingContext by wrapping the existing interactive shell @@ -333,7 +331,7 @@ for the full list of supported sources and artifacts.
To initialize a Spark Streaming program in Scala, a -[`StreamingContext`](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) +[`StreamingContext`](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object has to be created, which is the main entry point of all Spark Streaming functionality. A `StreamingContext` object can be created by using @@ -344,7 +342,7 @@ new StreamingContext(master, appName, batchDuration, [sparkHome], [jars])
To initialize a Spark Streaming program in Java, a -[`JavaStreamingContext`](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) +[`JavaStreamingContext`](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object has to be created, which is the main entry point of all Spark Streaming functionality. A `JavaStreamingContext` object can be created by using @@ -431,8 +429,8 @@ and process any files created in that directory. Note that For more details on streams from files, Akka actors and sockets, see the API documentations of the relevant functions in -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) for -Scala and [JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) +[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) for +Scala and [JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) for Java. Additional functionality for creating DStreams from sources such as Kafka, Flume, and Twitter @@ -802,10 +800,10 @@ output operators are defined: The complete list of DStream operations is available in the API documentation. For the Scala API, -see [DStream](api/streaming/index.html#org.apache.spark.streaming.dstream.DStream) -and [PairDStreamFunctions](api/streaming/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). -For the Java API, see [JavaDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.dstream.DStream) -and [JavaPairDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaPairDStream). +see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) +and [PairDStreamFunctions](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). +For the Java API, see [JavaDStream](api/scala/index.html#org.apache.spark.streaming.api.java.dstream.DStream) +and [JavaPairDStream](api/scala/index.html#org.apache.spark.streaming.api.java.JavaPairDStream). Specifically for the Java API, see [Spark's Java programming guide](java-programming-guide.html) for more information. @@ -881,7 +879,7 @@ Cluster resources maybe under-utilized if the number of parallel tasks used in a computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the -[`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) +[`PairDStreamFunctions`](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) documentation), or set the [config property](configuration.html#spark-properties) `spark.default.parallelism` to change the default. @@ -925,7 +923,7 @@ A good approach to figure out the right batch size for your application is to te conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (either look for "Total delay" in Spark driver log4j logs, or use the -[StreamingListener](api/streaming/index.html#org.apache.spark.streaming.scheduler.StreamingListener) +[StreamingListener](api/scala/index.html#org.apache.spark.streaming.scheduler.StreamingListener) interface). If the delay is maintained to be comparable to the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it @@ -952,7 +950,7 @@ exception saying so. ## Monitoring Besides Spark's in-built [monitoring capabilities](monitoring.html), the progress of a Spark Streaming program can also be monitored using the [StreamingListener] -(api/streaming/index.html#org.apache.spark.scheduler.StreamingListener) interface, +(api/scala/index.html#org.apache.spark.scheduler.StreamingListener) interface, which allows you to get statistics of batch processing times, queueing delays, and total end-to-end delays. Note that this is still an experimental API and it is likely to be improved upon (i.e., more information reported) in the future. @@ -965,9 +963,9 @@ in Spark Streaming applications and achieving more consistent batch processing t * **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, -[StorageLevel.MEMORY_ONLY_SER](api/core/index.html#org.apache.spark.storage.StorageLevel$) for +[StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for DStream compared to -[StorageLevel.MEMORY_ONLY](api/core/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). +[StorageLevel.MEMORY_ONLY](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs higher serialization/deserialization overheads, it significantly reduces GC pauses. @@ -1244,15 +1242,15 @@ and output 30 after recovery. # Where to Go from Here * API documentation - - Main docs of StreamingContext and DStreams in [Scala](api/streaming/index.html#org.apache.spark.streaming.package) - and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package) + - Main docs of StreamingContext and DStreams in [Scala](api/scala/index.html#org.apache.spark.streaming.package) + and [Java](api/scala/index.html#org.apache.spark.streaming.api.java.package) - Additional docs for - [Kafka](api/external/kafka/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), - [Flume](api/external/flume/index.html#org.apache.spark.streaming.flume.FlumeUtils$), - [Twitter](api/external/twitter/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), - [ZeroMQ](api/external/zeromq/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and - [MQTT](api/external/mqtt/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) + [Kafka](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), + [Flume](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), + [Twitter](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), + [ZeroMQ](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and + [MQTT](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) * More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/streaming/examples) -* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming +* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming. diff --git a/docs/tuning.md b/docs/tuning.md index cc069f0e84b9c..78e10770a80b6 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -48,7 +48,7 @@ Spark automatically includes Kryo serializers for the many commonly-used core Sc in the AllScalaRegistrar from the [Twitter chill](https://github.com/twitter/chill) library. To register your own custom classes with Kryo, create a public class that extends -[`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the +[`org.apache.spark.serializer.KryoRegistrator`](api/scala/index.html#org.apache.spark.serializer.KryoRegistrator) and set the `spark.kryo.registrator` config property to point to it, as follows: {% highlight scala %} @@ -222,7 +222,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a (though you can control it through optional parameters to `SparkContext.textFile`, etc), and for distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest parent RDD's number of partitions. You can pass the level of parallelism as a second argument -(see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation), +(see the [`spark.PairRDDFunctions`](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation), or set the config property `spark.default.parallelism` to change the default. In general, we recommend 2-3 tasks per CPU core in your cluster. diff --git a/examples/pom.xml b/examples/pom.xml index 0b6212b5d1549..e1fc149d87f17 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -31,20 +31,6 @@ Spark Project Examples http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark @@ -124,6 +110,10 @@ commons-logging commons-logging + + org.jruby + jruby-complete + @@ -176,6 +166,11 @@ + + com.github.scopt + scopt_${scala.binary.version} + 3.2.0 + @@ -187,7 +182,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-assembly-${project.version}.jar + ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar *:* diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index e8e63d2745692..b5b438e9753a7 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -34,19 +34,19 @@ public static class Person implements Serializable { private String name; private int age; - String getName() { + public String getName() { return name; } - void setName(String name) { + public void setName(String name) { this.name = name; } - int getAge() { + public int getAge() { return age; } - void setAge(int age) { + public void setAge(int age) { this.age = age; } } diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java new file mode 100644 index 0000000000000..a94fa621dc328 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java @@ -0,0 +1,152 @@ +/* + * 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.streaming.examples; + +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.receiver.Receiver; +import scala.Tuple2; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.net.ConnectException; +import java.net.Socket; +import java.util.regex.Pattern; + +/** + * Custom Receiver that receives data over a socket. Received bytes is interpreted as + * text and \n delimited lines are considered as records. They are then counted and printed. + * + * Usage: JavaCustomReceiver + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and of the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run org.apache.spark.streaming.examples.JavaCustomReceiver local[2] localhost 9999` + */ + +public class JavaCustomReceiver extends Receiver { + private static final Pattern SPACE = Pattern.compile(" "); + + public static void main(String[] args) { + if (args.length < 3) { + System.err.println("Usage: JavaNetworkWordCount \n" + + "In local mode, should be 'local[n]' with n > 1"); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + // Create the context with a 1 second batch size + JavaStreamingContext ssc = new JavaStreamingContext(args[0], "JavaNetworkWordCount", + new Duration(1000), System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); + + // Create a input stream with the custom receiver on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + JavaDStream lines = ssc.receiverStream( + new JavaCustomReceiver(args[1], Integer.parseInt(args[2]))); + JavaDStream words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String x) { + return Lists.newArrayList(SPACE.split(x)); + } + }); + JavaPairDStream wordCounts = words.mapToPair( + new PairFunction() { + @Override public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }).reduceByKey(new Function2() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + }); + + wordCounts.print(); + ssc.start(); + ssc.awaitTermination(); + } + + // ============= Receiver code that receives data over a socket ============== + + String host = null; + int port = -1; + + public JavaCustomReceiver(String host_ , int port_) { + super(StorageLevel.MEMORY_AND_DISK_2()); + host = host_; + port = port_; + } + + public void onStart() { + // Start the thread that receives data over a connection + new Thread() { + @Override public void run() { + receive(); + } + }.start(); + } + + public void onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private void receive() { + Socket socket = null; + String userInput = null; + + try { + // connect to the server + socket = new Socket(host, port); + + BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + + // Until stopped or connection broken continue reading + while (!isStopped() && (userInput = reader.readLine()) != null) { + System.out.println("Received data '" + userInput + "'"); + store(userInput); + } + reader.close(); + socket.close(); + + // Restart in an attempt to connect again when server is active again + restart("Trying to connect again"); + } catch(ConnectException ce) { + // restart if could not connect to server + restart("Could not connect", ce); + } catch(Throwable t) { + restart("Error receiving data", t); + } + } +} + + diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index 7f68d451e9b31..0cc9d0ae1a08e 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -31,7 +31,7 @@ /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - * Usage: NetworkWordCount + * Usage: JavaNetworkWordCount * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. * and describe the TCP server that Spark Streaming would connect to receive data. * @@ -43,9 +43,6 @@ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); - private JavaNetworkWordCount() { - } - public static void main(String[] args) { if (args.length < 3) { System.err.println("Usage: JavaNetworkWordCount \n" + 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 c8c916bb45e00..f6dfd2c4c6217 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -35,7 +35,7 @@ object BroadcastTest { System.setProperty("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(args(0), "Broadcast Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index 4e787240e912d..3798329fc2f41 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -75,7 +75,7 @@ object CassandraCQLTest { val sc = new SparkContext(args(0), "CQLTestApp", System.getenv("SPARK_HOME"), - SparkContext.jarOfClass(this.getClass)) + SparkContext.jarOfClass(this.getClass).toSeq) val cHost: String = args(1) val cPort: String = args(2) val KeySpace = "retail" diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index be7d39549a28d..f0dcef431b2e1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -27,7 +27,7 @@ object ExceptionHandlingTest { } val sc = new SparkContext(args(0), "ExceptionHandlingTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) sc.parallelize(0 until sc.defaultParallelism).foreach { i => if (math.random > 0.75) { throw new Exception("Testing exception handling") diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 2b7ecdc991325..e67bb29a49405 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -36,7 +36,7 @@ object GroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index cbf78e8e9eba1..adbd1c02fa2ea 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -27,7 +27,7 @@ import org.apache.spark.rdd.NewHadoopRDD object HBaseTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HBaseTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val conf = HBaseConfiguration.create() diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index dd6d5205133be..c7a4884af10b7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -22,7 +22,7 @@ import org.apache.spark._ object HdfsTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HdfsTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val file = sc.textFile(args(1)) val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 35758fa003d94..f77a444ff7a9f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -46,11 +46,10 @@ object LogQuery { } val sc = new SparkContext(args(0), "Log Query", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val dataSet = - if (args.length == 2) sc.textFile(args(1)) - else sc.parallelize(exampleApacheLogs) + if (args.length == 2) sc.textFile(args(1)) else sc.parallelize(exampleApacheLogs) // scalastyle:off val apacheLogRegex = """^([\d.]+) (\S+) (\S+) \[([\w\d:/]+\s[+\-]\d{4})\] "(.+?)" (\d{3}) ([\d\-]+) "([^"]+)" "([^"]+)".*""".r diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 58f26f1e24052..c8985eae33de3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -28,7 +28,7 @@ object MultiBroadcastTest { } val sc = new SparkContext(args(0), "Multi-Broadcast Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 557a0c1841339..54e8503711e30 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -37,7 +37,7 @@ object SimpleSkewedGroupByTest { var ratio = if (args.length > 5) args(5).toInt else 5.0 val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 05a74725b875b..1c5f22e1c00bb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -36,7 +36,7 @@ object SkewedGroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 191c82fd913ee..0dc726aecdd28 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -113,7 +113,7 @@ object SparkALS { printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) val sc = new SparkContext(host, "SparkALS", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index fd63ba3dbce7d..3a6f18c33ea4b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -56,7 +56,7 @@ object SparkHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq, Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 8aa31d7e6a2c2..dcae9591b0407 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -57,7 +57,7 @@ object SparkKMeans { System.exit(1) } val sc = new SparkContext(args(0), "SparkLocalKMeans", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val lines = sc.textFile(args(1)) val data = lines.map(parseVector _).cache() val K = args(2).toInt diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index d70ce603bb71d..4f74882ccbea5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -52,7 +52,7 @@ object SparkLR { System.exit(1) } val sc = new SparkContext(args(0), "SparkLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val numSlices = if (args.length > 1) args(1).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 60e4a11a21f69..fa41c5c560943 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -37,7 +37,7 @@ object SparkPageRank { } var iters = args(2).toInt val ctx = new SparkContext(args(0), "PageRank", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val lines = ctx.textFile(args(1), 1) val links = lines.map{ s => val parts = s.split("\\s+") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index d3babc3ed12c8..d8f5720504223 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -29,7 +29,7 @@ object SparkPi { System.exit(1) } val spark = new SparkContext(args(0), "SparkPi", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices val count = spark.parallelize(1 to n, slices).map { i => diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 65bd61abda6cd..17d983cd875db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -47,7 +47,7 @@ object SparkTC { System.exit(1) } val spark = new SparkContext(args(0), "SparkTC", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 var tc = spark.parallelize(generateGraph, slices).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 4f558929add51..7e43c384bdb9d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -58,7 +58,7 @@ object SparkTachyonHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkTachyonHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq, Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index 2b207fd8d3e16..93459110e4e0e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -33,7 +33,7 @@ object SparkTachyonPi { System.exit(1) } val spark = new SparkContext(args(0), "SparkTachyonPi", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala new file mode 100644 index 0000000000000..ec9de022c1d47 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -0,0 +1,145 @@ +/* + * 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.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.classification.{LogisticRegressionWithSGD, SVMWithSGD} +import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} + +/** + * An example app for binary classification. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.BinaryClassification + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object BinaryClassification { + + object Algorithm extends Enumeration { + type Algorithm = Value + val SVM, LR = Value + } + + object RegType extends Enumeration { + type RegType = Value + val L1, L2 = Value + } + + import Algorithm._ + import RegType._ + + case class Params( + input: String = null, + numIterations: Int = 100, + stepSize: Double = 1.0, + algorithm: Algorithm = LR, + regType: RegType = L2, + regParam: Double = 0.1) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("BinaryClassification") { + head("BinaryClassification: an example app for binary classification.") + opt[Int]("numIterations") + .text("number of iterations") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("stepSize") + .text(s"initial step size, default: ${defaultParams.stepSize}") + .action((x, c) => c.copy(stepSize = x)) + opt[String]("algorithm") + .text(s"algorithm (${Algorithm.values.mkString(",")}), " + + s"default: ${defaultParams.algorithm}") + .action((x, c) => c.copy(algorithm = Algorithm.withName(x))) + opt[String]("regType") + .text(s"regularization type (${RegType.values.mkString(",")}), " + + s"default: ${defaultParams.regType}") + .action((x, c) => c.copy(regType = RegType.withName(x))) + opt[Double]("regParam") + .text(s"regularization parameter, default: ${defaultParams.regParam}") + arg[String]("") + .required() + .text("input paths to labeled examples in LIBSVM format") + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"BinaryClassification with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val examples = MLUtils.loadLibSVMData(sc, params.input).cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + println(s"Training: $numTraining, test: $numTest.") + + examples.unpersist(blocking = false) + + val updater = params.regType match { + case L1 => new L1Updater() + case L2 => new SquaredL2Updater() + } + + val model = params.algorithm match { + case LR => + val algorithm = new LogisticRegressionWithSGD() + algorithm.optimizer + .setNumIterations(params.numIterations) + .setStepSize(params.stepSize) + .setUpdater(updater) + .setRegParam(params.regParam) + algorithm.run(training).clearThreshold() + case SVM => + val algorithm = new SVMWithSGD() + algorithm.optimizer + .setNumIterations(params.numIterations) + .setStepSize(params.stepSize) + .setUpdater(updater) + .setRegParam(params.regParam) + algorithm.run(training).clearThreshold() + } + + val prediction = model.predict(test.map(_.features)) + val predictionAndLabel = prediction.zip(test.map(_.label)) + + val metrics = new BinaryClassificationMetrics(predictionAndLabel) + + println(s"Test areaUnderPR = ${metrics.areaUnderPR()}.") + println(s"Test areaUnderROC = ${metrics.areaUnderROC()}.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala new file mode 100644 index 0000000000000..0bd847d7bab30 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -0,0 +1,161 @@ +/* + * 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 scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree, impurity} +import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.model.DecisionTreeModel +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD + +/** + * An example runner for decision tree. Run with + * {{{ + * ./bin/spark-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DecisionTreeRunner { + + object ImpurityType extends Enumeration { + type ImpurityType = Value + val Gini, Entropy, Variance = Value + } + + import ImpurityType._ + + case class Params( + input: String = null, + algo: Algo = Classification, + maxDepth: Int = 5, + impurity: ImpurityType = Gini, + maxBins: Int = 20) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DecisionTreeRunner") { + head("DecisionTreeRunner: an example decision tree app.") + opt[String]("algo") + .text(s"algorithm (${Algo.values.mkString(",")}), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algo = Algo.withName(x))) + opt[String]("impurity") + .text(s"impurity type (${ImpurityType.values.mkString(",")}), " + + s"default: ${defaultParams.impurity}") + .action((x, c) => c.copy(impurity = ImpurityType.withName(x))) + opt[Int]("maxDepth") + .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") + .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("maxBins") + .text(s"max number of bins, default: ${defaultParams.maxBins}") + .action((x, c) => c.copy(maxBins = x)) + arg[String]("") + .text("input paths to labeled examples in dense format (label,f0 f1 f2 ...)") + .required() + .action((x, c) => c.copy(input = x)) + checkConfig { params => + if (params.algo == Classification && + (params.impurity == Gini || params.impurity == Entropy)) { + success + } else if (params.algo == Regression && params.impurity == Variance) { + success + } else { + failure(s"Algo ${params.algo} is not compatible with impurity ${params.impurity}.") + } + } + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName("DecisionTreeRunner") + val sc = new SparkContext(conf) + + // Load training data and cache it. + val examples = MLUtils.loadLabeledData(sc, params.input).cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + + println(s"numTraining = $numTraining, numTest = $numTest.") + + examples.unpersist(blocking = false) + + val impurityCalculator = params.impurity match { + case Gini => impurity.Gini + case Entropy => impurity.Entropy + case Variance => impurity.Variance + } + + val strategy = new Strategy(params.algo, impurityCalculator, params.maxDepth, params.maxBins) + val model = DecisionTree.train(training, strategy) + + if (params.algo == Classification) { + val accuracy = accuracyScore(model, test) + println(s"Test accuracy = $accuracy.") + } + + if (params.algo == Regression) { + val mse = meanSquaredError(model, test) + println(s"Test mean squared error = $mse.") + } + + sc.stop() + } + + /** + * Calculates the classifier accuracy. + */ + private def accuracyScore( + model: DecisionTreeModel, + data: RDD[LabeledPoint], + threshold: Double = 0.5): Double = { + def predictedValue(features: Vector): Double = { + if (model.predict(features) < threshold) 0.0 else 1.0 + } + val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() + val count = data.count() + correctCount.toDouble / count + } + + /** + * Calculates the mean squared error for regression. + */ + private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { + data.map { y => + val err = tree.predict(y.features) - y.label + err * err + }.mean() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala new file mode 100644 index 0000000000000..f96bc1bf00b92 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala @@ -0,0 +1,109 @@ +/* + * 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.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.clustering.KMeans +import org.apache.spark.mllib.linalg.Vectors + +/** + * An example k-means app. Run with + * {{{ + * ./bin/spark-example org.apache.spark.examples.mllib.DenseKMeans [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DenseKMeans { + + object InitializationMode extends Enumeration { + type InitializationMode = Value + val Random, Parallel = Value + } + + import InitializationMode._ + + case class Params( + input: String = null, + k: Int = -1, + numIterations: Int = 10, + initializationMode: InitializationMode = Parallel) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DenseKMeans") { + head("DenseKMeans: an example k-means app for dense data.") + opt[Int]('k', "k") + .required() + .text(s"number of clusters, required") + .action((x, c) => c.copy(k = x)) + opt[Int]("numIterations") + .text(s"number of iterations, default; ${defaultParams.numIterations}") + .action((x, c) => c.copy(numIterations = x)) + opt[String]("initMode") + .text(s"initialization mode (${InitializationMode.values.mkString(",")}), " + + s"default: ${defaultParams.initializationMode}") + .action((x, c) => c.copy(initializationMode = InitializationMode.withName(x))) + arg[String]("") + .text("input paths to examples") + .required() + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"DenseKMeans with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val examples = sc.textFile(params.input).map { line => + Vectors.dense(line.split(' ').map(_.toDouble)) + }.cache() + + val numExamples = examples.count() + + println(s"numExamples = $numExamples.") + + val initMode = params.initializationMode match { + case Random => KMeans.RANDOM + case Parallel => KMeans.K_MEANS_PARALLEL + } + + val model = new KMeans() + .setInitializationMode(initMode) + .setK(params.k) + .setMaxIterations(params.numIterations) + .run(examples) + + val cost = model.computeCost(examples) + + println(s"Total cost = $cost.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala new file mode 100644 index 0000000000000..1723ca6931021 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -0,0 +1,125 @@ +/* + * 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.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.mllib.util.{MulticlassLabelParser, MLUtils} +import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1Updater} + +/** + * An example app for linear regression. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.LinearRegression + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object LinearRegression extends App { + + object RegType extends Enumeration { + type RegType = Value + val NONE, L1, L2 = Value + } + + import RegType._ + + case class Params( + input: String = null, + numIterations: Int = 100, + stepSize: Double = 1.0, + regType: RegType = L2, + regParam: Double = 0.1) + + val defaultParams = Params() + + val parser = new OptionParser[Params]("LinearRegression") { + head("LinearRegression: an example app for linear regression.") + opt[Int]("numIterations") + .text("number of iterations") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("stepSize") + .text(s"initial step size, default: ${defaultParams.stepSize}") + .action((x, c) => c.copy(stepSize = x)) + opt[String]("regType") + .text(s"regularization type (${RegType.values.mkString(",")}), " + + s"default: ${defaultParams.regType}") + .action((x, c) => c.copy(regType = RegType.withName(x))) + opt[Double]("regParam") + .text(s"regularization parameter, default: ${defaultParams.regParam}") + arg[String]("") + .required() + .text("input paths to labeled examples in LIBSVM format") + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"LinearRegression with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser).cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + println(s"Training: $numTraining, test: $numTest.") + + examples.unpersist(blocking = false) + + val updater = params.regType match { + case NONE => new SimpleUpdater() + case L1 => new L1Updater() + case L2 => new SquaredL2Updater() + } + + val algorithm = new LinearRegressionWithSGD() + algorithm.optimizer + .setNumIterations(params.numIterations) + .setStepSize(params.stepSize) + .setUpdater(updater) + .setRegParam(params.regParam) + + val model = algorithm.run(training) + + val prediction = model.predict(test.map(_.features)) + val predictionAndLabel = prediction.zip(test.map(_.label)) + + val loss = predictionAndLabel.map { case (p, l) => + val err = p - l + err * err + }.reduce(_ + _) + val rmse = math.sqrt(loss / numTest) + + println(s"Test RMSE = $rmse.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala new file mode 100644 index 0000000000000..703f02255b94b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -0,0 +1,131 @@ +/* + * 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 com.esotericsoftware.kryo.Kryo +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.recommendation.{ALS, MatrixFactorizationModel, Rating} +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.{KryoSerializer, KryoRegistrator} + +/** + * An example app for ALS on MovieLens data (http://grouplens.org/datasets/movielens/). + */ +object MovieLensALS { + + class ALSRegistrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[Rating]) + } + } + + case class Params( + input: String = null, + kryo: Boolean = false, + numIterations: Int = 20, + lambda: Double = 1.0, + rank: Int = 10) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("MovieLensALS") { + head("MovieLensALS: an example app for ALS on MovieLens data.") + opt[Int]("rank") + .text(s"rank, default: ${defaultParams.rank}}") + .action((x, c) => c.copy(rank = x)) + opt[Int]("numIterations") + .text(s"number of iterations, default: ${defaultParams.numIterations}") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("lambda") + .text(s"lambda (smoothing constant), default: ${defaultParams.lambda}") + .action((x, c) => c.copy(lambda = x)) + opt[Unit]("kryo") + .text(s"use Kryo serialization") + .action((_, c) => c.copy(kryo = true)) + arg[String]("") + .required() + .text("input paths to a MovieLens dataset of ratings") + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + System.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"MovieLensALS with $params") + if (params.kryo) { + conf.set("spark.serializer", classOf[KryoSerializer].getName) + .set("spark.kryo.registrator", classOf[ALSRegistrator].getName) + .set("spark.kryoserializer.buffer.mb", "8") + } + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val ratings = sc.textFile(params.input).map { line => + val fields = line.split("::") + Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) + }.cache() + + val numRatings = ratings.count() + val numUsers = ratings.map(_.user).distinct().count() + val numMovies = ratings.map(_.product).distinct().count() + + println(s"Got $numRatings ratings from $numUsers users on $numMovies movies.") + + val splits = ratings.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + println(s"Training: $numTraining, test: $numTest.") + + ratings.unpersist(blocking = false) + + val model = new ALS() + .setRank(params.rank) + .setIterations(params.numIterations) + .setLambda(params.lambda) + .run(training) + + val rmse = computeRmse(model, test, numTest) + + println(s"Test RMSE = $rmse.") + + sc.stop() + } + + /** Compute RMSE (Root Mean Squared Error). */ + def computeRmse(model: MatrixFactorizationModel, data: RDD[Rating], n: Long) = { + val predictions: RDD[Rating] = model.predict(data.map(x => (x.user, x.product))) + val predictionsAndRatings = predictions.map(x => ((x.user, x.product), x.rating)) + .join(data.map(x => ((x.user, x.product), x.rating))) + .values + math.sqrt(predictionsAndRatings.map(x => (x._1 - x._2) * (x._1 - x._2)).reduce(_ + _) / n) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala new file mode 100644 index 0000000000000..25b6768b8d72b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -0,0 +1,102 @@ +/* + * 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.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.classification.NaiveBayes +import org.apache.spark.mllib.util.{MLUtils, MulticlassLabelParser} + +/** + * An example naive Bayes app. Run with + * {{{ + * ./bin/spark-example org.apache.spark.examples.mllib.SparseNaiveBayes [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object SparseNaiveBayes { + + case class Params( + input: String = null, + minPartitions: Int = 0, + numFeatures: Int = -1, + lambda: Double = 1.0) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("SparseNaiveBayes") { + head("SparseNaiveBayes: an example naive Bayes app for LIBSVM data.") + opt[Int]("numPartitions") + .text("min number of partitions") + .action((x, c) => c.copy(minPartitions = x)) + opt[Int]("numFeatures") + .text("number of features") + .action((x, c) => c.copy(numFeatures = x)) + opt[Double]("lambda") + .text(s"lambda (smoothing constant), default: ${defaultParams.lambda}") + .action((x, c) => c.copy(lambda = x)) + arg[String]("") + .text("input paths to labeled examples in LIBSVM format") + .required() + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"SparseNaiveBayes with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val minPartitions = + if (params.minPartitions > 0) params.minPartitions else sc.defaultMinPartitions + + val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser, + params.numFeatures, minPartitions) + // Cache examples because it will be used in both training and evaluation. + examples.cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0) + val test = splits(1) + + val numTraining = training.count() + val numTest = test.count() + + println(s"numTraining = $numTraining, numTest = $numTest.") + + val model = new NaiveBayes().setLambda(params.lambda).run(training) + + val prediction = model.predict(test.map(_.features)) + val predictionAndLabel = prediction.zip(test.map(_.label)) + val accuracy = predictionAndLabel.filter(x => x._1 == x._2).count().toDouble / numTest + + println(s"Test accuracy = $accuracy.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index 61b9655cd3759..3cd9cb743e309 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -35,20 +35,16 @@ import org.apache.spark.mllib.linalg.Vectors */ object TallSkinnyPCA { def main(args: Array[String]) { - if (args.length != 2) { - System.err.println("Usage: TallSkinnyPCA ") + if (args.length != 1) { + System.err.println("Usage: TallSkinnyPCA ") System.exit(1) } - val conf = new SparkConf() - .setMaster(args(0)) - .setAppName("TallSkinnyPCA") - .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass)) + val conf = new SparkConf().setAppName("TallSkinnyPCA") val sc = new SparkContext(conf) // Load and parse the data file. - val rows = sc.textFile(args(1)).map { line => + val rows = sc.textFile(args(0)).map { line => val values = line.split(' ').map(_.toDouble) Vectors.dense(values) } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 9aeebf58eabfb..4d6690318615a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -35,20 +35,16 @@ import org.apache.spark.mllib.linalg.Vectors */ object TallSkinnySVD { def main(args: Array[String]) { - if (args.length != 2) { - System.err.println("Usage: TallSkinnySVD ") + if (args.length != 1) { + System.err.println("Usage: TallSkinnySVD ") System.exit(1) } - val conf = new SparkConf() - .setMaster(args(0)) - .setAppName("TallSkinnySVD") - .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass)) + val conf = new SparkConf().setAppName("TallSkinnySVD") val sc = new SparkContext(conf) // Load and parse the data file. - val rows = sc.textFile(args(1)).map { line => + val rows = sc.textFile(args(0)).map { line => val values = line.split(' ').map(_.toDouble) Vectors.dense(values) } diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index a22e64ca3ce45..c845dd8904c90 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -26,8 +26,8 @@ import akka.actor.{Actor, ActorRef, Props, actorRef2Scala} import org.apache.spark.{SparkConf, SecurityManager} import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions -import org.apache.spark.streaming.receivers.Receiver import org.apache.spark.util.AkkaUtils +import org.apache.spark.streaming.receiver.ActorHelper case class SubscribeReceiver(receiverActor: ActorRef) case class UnsubscribeReceiver(receiverActor: ActorRef) @@ -81,14 +81,14 @@ class FeederActor extends Actor { * @see [[org.apache.spark.streaming.examples.FeederActor]] */ class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) -extends Actor with Receiver { +extends Actor with ActorHelper { lazy private val remotePublisher = context.actorSelection(urlOfPublisher) override def preStart = remotePublisher ! SubscribeReceiver(context.self) def receive = { - case msg => pushBlock(msg.asInstanceOf[T]) + case msg => store(msg.asInstanceOf[T]) } override def postStop() = remotePublisher ! UnsubscribeReceiver(context.self) @@ -150,7 +150,7 @@ object ActorWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) /* * Following is the use of actorStream to plug in custom actor as receiver diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala new file mode 100644 index 0000000000000..eebffd824983f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala @@ -0,0 +1,108 @@ +/* + * 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.streaming.examples + +import java.io.{InputStreamReader, BufferedReader, InputStream} +import java.net.Socket + +import org.apache.spark.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 + +/** + * Custom Receiver that receives data over a socket. Received bytes is interpreted as + * text and \n delimited lines are considered as records. They are then counted and printed. + * + * Usage: CustomReceiver + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and of the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run org.apache.spark.streaming.examples.CustomReceiver local[2] localhost 9999` + */ +object CustomReceiver { + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: NetworkWordCount \n" + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + // Create the context with a 1 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + + // Create a input stream with the custom receiver on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + val lines = ssc.receiverStream(new CustomReceiver(args(1), args(2).toInt)) + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) + wordCounts.print() + ssc.start() + ssc.awaitTermination() + } +} + + +class CustomReceiver(host: String, port: Int) + extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) with Logging { + + def onStart() { + // Start the thread that receives data over a connection + new Thread("Socket Receiver") { + override def run() { receive() } + }.start() + } + + def onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private def receive() { + var socket: Socket = null + var userInput: String = null + try { + logInfo("Connecting to " + host + ":" + port) + socket = new Socket(host, port) + logInfo("Connected to " + host + ":" + port) + val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) + userInput = reader.readLine() + while(!isStopped && userInput != null) { + store(userInput) + userInput = reader.readLine() + } + reader.close() + socket.close() + logInfo("Stopped receiving") + restart("Trying to connect again") + } catch { + case e: java.net.ConnectException => + restart("Error connecting to " + host + ":" + port, e) + case t: Throwable => + restart("Error receiving data", t) + } + } +} diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 11c3aaad3c8a8..26b6024534124 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -51,7 +51,7 @@ object FlumeEventCount { val batchInterval = Milliseconds(2000) // Create the context and set the batch size val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a flume stream val stream = FlumeUtils.createStream(ssc, host,port,StorageLevel.MEMORY_ONLY_SER_2) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 1c0ce3111e290..7f86fc792aacf 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -41,7 +41,7 @@ object HdfsWordCount { // Create the context val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create the FileInputDStream on the directory and use the // stream to count words in new files created diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index cca0be2cbb9c9..2aa4f1474a59e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -52,7 +52,7 @@ object KafkaWordCount { val Array(master, zkQuorum, group, topics, numThreads) = args val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap @@ -71,7 +71,7 @@ object KafkaWordCount { object KafkaWordCountProducer { def main(args: Array[String]) { - if (args.length < 2) { + if (args.length < 4) { System.err.println("Usage: KafkaWordCountProducer " + " ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index 656222e0c1b31..62aef0fb47107 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -56,7 +56,7 @@ object MQTTPublisher { val msg: String = "hello mqtt demo for spark streaming" while (true) { - val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes()) + val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes("utf-8")) msgtopic.publish(message) println("Published data. topic: " + msgtopic.getName() + " Message: " + message) } @@ -97,7 +97,7 @@ object MQTTWordCount { val Seq(master, brokerUrl, topic) = args.toSeq val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), - StreamingContext.jarOfClass(this.getClass)) + StreamingContext.jarOfClass(this.getClass).toSeq) val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) val words = lines.flatMap(x => x.toString.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index d4c4d86b3466c..272ab11212451 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -47,7 +47,7 @@ object NetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index 612ecf7b7821a..ff2a205ec1c15 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -35,7 +35,7 @@ object QueueStream { // Create the context val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala index cdd7547d0d3b4..d915c0c39b334 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala @@ -50,7 +50,7 @@ object RawNetworkGrep { // Create the context val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Warm up the JVMs on master and slave for JIT compilation to kick in RawTextHelper.warmUp(ssc.sparkContext) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala index 56d10a964b71b..4aacbb1991418 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala @@ -77,7 +77,7 @@ object RecoverableNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(master, "RecoverableNetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index 14f65a2f8d46c..ef94c9298dd93 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -54,7 +54,7 @@ object StatefulNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", - Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index 8a654f8fada21..c38905e8f3663 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -68,7 +68,7 @@ object TwitterAlgebirdCMS { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index 45771d7050eeb..c067046f9020e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -55,7 +55,7 @@ object TwitterAlgebirdHLL { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 896d010c68f18..2597c8178862a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -41,7 +41,7 @@ object TwitterPopularTags { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) val stream = TwitterUtils.createStream(ssc, None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 6f88db1abf19d..109ff855b5228 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -86,7 +86,7 @@ object ZeroMQWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 2b130fb30eaaa..673013f7cf948 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -45,7 +45,7 @@ object PageViewStream { // Create the context val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a NetworkInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.socketTextStream(host, port) @@ -69,8 +69,11 @@ object PageViewStream { val normalCount = statuses.filter(_ == 200).size val errorCount = statuses.size - normalCount val errorRatio = errorCount.toFloat / statuses.size - if (errorRatio > 0.05) {"%s: **%s**".format(zip, errorRatio)} - else {"%s: %s".format(zip, errorRatio)} + if (errorRatio > 0.05) { + "%s: **%s**".format(zip, errorRatio) + } else { + "%s: %s".format(zip, errorRatio) + } } // Return the number unique users in last 15 seconds diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f21963531574b..03d3b2394f510 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -31,20 +31,6 @@ Spark Project External Flume http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark @@ -61,12 +47,16 @@ org.apache.flume flume-ng-sdk - 1.2.0 + 1.4.0 org.jboss.netty netty + + org.apache.thrift + libthrift + diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 34012b846e21e..df7605fe579f8 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -34,6 +34,8 @@ import org.apache.spark.util.Utils import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ +import org.apache.spark.Logging +import org.apache.spark.streaming.receiver.Receiver private[streaming] class FlumeInputDStream[T: ClassTag]( @@ -41,9 +43,9 @@ class FlumeInputDStream[T: ClassTag]( host: String, port: Int, storageLevel: StorageLevel -) extends NetworkInputDStream[SparkFlumeEvent](ssc_) { +) extends ReceiverInputDStream[SparkFlumeEvent](ssc_) { - override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = { + override def getReceiver(): Receiver[SparkFlumeEvent] = { new FlumeReceiver(host, port, storageLevel) } } @@ -115,13 +117,13 @@ private[streaming] object SparkFlumeEvent { private[streaming] class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { override def append(event : AvroFlumeEvent) : Status = { - receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event) + receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event)) Status.OK } override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = { events.foreach (event => - receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event)) + receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) Status.OK } } @@ -133,23 +135,21 @@ class FlumeReceiver( host: String, port: Int, storageLevel: StorageLevel - ) extends NetworkReceiver[SparkFlumeEvent] { + ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { - lazy val blockGenerator = new BlockGenerator(storageLevel) + lazy val responder = new SpecificResponder( + classOf[AvroSourceProtocol], new FlumeEventServer(this)) + lazy val server = new NettyServer(responder, new InetSocketAddress(host, port)) - protected override def onStart() { - val responder = new SpecificResponder( - classOf[AvroSourceProtocol], new FlumeEventServer(this)) - val server = new NettyServer(responder, new InetSocketAddress(host, port)) - blockGenerator.start() + def onStart() { server.start() logInfo("Flume receiver started") } - protected override def onStop() { - blockGenerator.stop() + def onStop() { + server.close() logInfo("Flume receiver stopped") } - override def getLocationPreference = Some(host) + override def preferredLocation = Some(host) } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 654ba451e72fb..499f3560ef768 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -19,8 +19,8 @@ package org.apache.spark.streaming.flume import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaInputDStream, JavaStreamingContext, JavaDStream} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object FlumeUtils { /** @@ -35,7 +35,7 @@ object FlumeUtils { hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[SparkFlumeEvent] = { + ): ReceiverInputDStream[SparkFlumeEvent] = { val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel) inputStream } @@ -50,7 +50,7 @@ object FlumeUtils { jssc: JavaStreamingContext, hostname: String, port: Int - ): JavaDStream[SparkFlumeEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createStream(jssc.ssc, hostname, port) } @@ -65,7 +65,7 @@ object FlumeUtils { hostname: String, port: Int, storageLevel: StorageLevel - ): JavaDStream[SparkFlumeEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createStream(jssc.ssc, hostname, port, storageLevel) } } diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java index 733389b98d22d..e0ad4f1015205 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -19,16 +19,16 @@ import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.junit.Test; public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { @Test public void testFlumeStream() { // tests the API, does not actually test data receiving - JavaDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); - JavaDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, + JavaReceiverInputDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); + JavaReceiverInputDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 8bc43972ab6a0..dd287d0ef90a0 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -31,6 +31,7 @@ import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase} import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream class FlumeStreamSuite extends TestSuiteBase { @@ -39,10 +40,11 @@ class FlumeStreamSuite extends TestSuiteBase { test("flume input stream") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream = FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) + val flumeStream: JavaReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) + val outputStream = new TestOutputStream(flumeStream.receiverInputDStream, outputBuffer) outputStream.register() ssc.start() @@ -55,7 +57,7 @@ class FlumeStreamSuite extends TestSuiteBase { for (i <- 0 until input.size) { val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(input(i).toString.getBytes())) + event.setBody(ByteBuffer.wrap(input(i).toString.getBytes("utf-8"))) event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header")) client.append(event) Thread.sleep(500) diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 343e1fabd823f..979eb0ca624bd 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -31,20 +31,6 @@ Spark Project External Kafka http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index c2d9dcbfaac7a..21443ebbbfb0e 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -33,6 +33,7 @@ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver /** * Input stream that pulls messages from a Kafka Broker. @@ -53,11 +54,11 @@ class KafkaInputDStream[ kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ) extends NetworkInputDStream[(K, V)](ssc_) with Logging { + ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { - def getReceiver(): NetworkReceiver[(K, V)] = { + def getReceiver(): Receiver[(K, V)] = { new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) - .asInstanceOf[NetworkReceiver[(K, V)]] + .asInstanceOf[Receiver[(K, V)]] } } @@ -70,21 +71,15 @@ class KafkaReceiver[ kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ) extends NetworkReceiver[Any] { + ) extends Receiver[Any](storageLevel) with Logging { - // Handles pushing data into the BlockManager - lazy protected val blockGenerator = new BlockGenerator(storageLevel) // Connection to Kafka var consumerConnector : ConsumerConnector = null - def onStop() { - blockGenerator.stop() - } + def onStop() { } def onStart() { - blockGenerator.start() - // In case we are using multiple Threads to handle Kafka Messages val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) @@ -130,7 +125,7 @@ class KafkaReceiver[ def run() { logInfo("Starting MessageHandler.") for (msgAndMetadata <- stream) { - blockGenerator += (msgAndMetadata.key, msgAndMetadata.message) + store((msgAndMetadata.key, msgAndMetadata.message)) } } } 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 5472d0cd04a94..86bb91f362d29 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 @@ -27,8 +27,8 @@ import kafka.serializer.{Decoder, StringDecoder} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaPairReceiverInputDStream, JavaStreamingContext, JavaPairDStream} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object KafkaUtils { @@ -48,7 +48,7 @@ object KafkaUtils { groupId: String, topics: Map[String, Int], storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[(String, String)] = { + ): ReceiverInputDStream[(String, String)] = { val kafkaParams = Map[String, String]( "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, "zookeeper.connection.timeout.ms" -> "10000") @@ -70,7 +70,7 @@ object KafkaUtils { kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ): DStream[(K, V)] = { + ): ReceiverInputDStream[(K, V)] = { new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel) } @@ -88,7 +88,7 @@ object KafkaUtils { zkQuorum: String, groupId: String, topics: JMap[String, JInt] - ): JavaPairDStream[String, String] = { + ): JavaPairReceiverInputDStream[String, String] = { implicit val cmt: ClassTag[String] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) @@ -110,7 +110,7 @@ object KafkaUtils { groupId: String, topics: JMap[String, JInt], storageLevel: StorageLevel - ): JavaPairDStream[String, String] = { + ): JavaPairReceiverInputDStream[String, String] = { implicit val cmt: ClassTag[String] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), @@ -139,7 +139,7 @@ object KafkaUtils { kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel - ): JavaPairDStream[K, V] = { + ): JavaPairReceiverInputDStream[K, V] = { implicit val keyCmt: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val valueCmt: ClassTag[V] = diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 7b4999447ee69..9f8046bf00f8f 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -18,12 +18,13 @@ package org.apache.spark.streaming.kafka; import java.util.HashMap; + +import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream; import org.junit.Test; import com.google.common.collect.Maps; import kafka.serializer.StringDecoder; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaPairDStream; public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { @Test @@ -31,14 +32,15 @@ public void testKafkaStream() { HashMap topics = Maps.newHashMap(); // tests the API, does not actually test data receiving - JavaPairDStream test1 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); - JavaPairDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, + JavaPairReceiverInputDStream test1 = + KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); + JavaPairReceiverInputDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2()); HashMap kafkaParams = Maps.newHashMap(); kafkaParams.put("zookeeper.connect", "localhost:12345"); kafkaParams.put("group.id","consumer-group"); - JavaPairDStream test3 = KafkaUtils.createStream(ssc, + JavaPairReceiverInputDStream test3 = KafkaUtils.createStream(ssc, String.class, String.class, StringDecoder.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index d9809f6409d44..e6f2c4a5cf5d1 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.kafka import kafka.serializer.StringDecoder import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.ReceiverInputDStream class KafkaStreamSuite extends TestSuiteBase { @@ -28,10 +29,13 @@ class KafkaStreamSuite extends TestSuiteBase { val topics = Map("my-topic" -> 1) // tests the API, does not actually test data receiving - val test1 = KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) - val test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1: ReceiverInputDStream[(String, String)] = + KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) + val test2: ReceiverInputDStream[(String, String)] = + KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group") - val test3 = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + val test3: ReceiverInputDStream[(String, String)] = + KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 3710a63541d78..9aa1c1a9f5b80 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -31,20 +31,6 @@ Spark Project External MQTT http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 1204cfba39f77..77661f71ada21 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -39,6 +39,7 @@ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver /** * Input stream that subscribe messages from a Mqtt Broker. @@ -49,38 +50,36 @@ import org.apache.spark.streaming.dstream._ */ private[streaming] -class MQTTInputDStream[T: ClassTag]( +class MQTTInputDStream( @transient ssc_ : StreamingContext, brokerUrl: String, topic: String, storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_) with Logging { - - def getReceiver(): NetworkReceiver[T] = { - new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]] + ) extends ReceiverInputDStream[String](ssc_) with Logging { + + def getReceiver(): Receiver[String] = { + new MQTTReceiver(brokerUrl, topic, storageLevel) } } -private[streaming] -class MQTTReceiver(brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends NetworkReceiver[Any] { - lazy protected val blockGenerator = new BlockGenerator(storageLevel) +private[streaming] +class MQTTReceiver( + brokerUrl: String, + topic: String, + storageLevel: StorageLevel + ) extends Receiver[String](storageLevel) { def onStop() { - blockGenerator.stop() - } + } + def onStart() { - blockGenerator.start() - - // Set up persistence for messages - var peristance: MqttClientPersistence = new MemoryPersistence() + // Set up persistence for messages + val persistence = new MemoryPersistence() // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance - var client: MqttClient = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance) + val client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) // Connect to MqttBroker client.connect() @@ -89,18 +88,18 @@ class MQTTReceiver(brokerUrl: String, client.subscribe(topic) // Callback automatically triggers as and when new message arrives on specified topic - var callback: MqttCallback = new MqttCallback() { + val callback: MqttCallback = new MqttCallback() { // Handles Mqtt message override def messageArrived(arg0: String, arg1: MqttMessage) { - blockGenerator += new String(arg1.getPayload()) + store(new String(arg1.getPayload(),"utf-8")) } override def deliveryComplete(arg0: IMqttDeliveryToken) { } override def connectionLost(arg0: Throwable) { - logInfo("Connection lost " + arg0) + restart("Connection lost ", arg0) } } diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala index 1b09ee5dc8f65..c5ffe51f9986c 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala @@ -19,9 +19,9 @@ package org.apache.spark.streaming.mqtt import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext, JavaDStream} import scala.reflect.ClassTag -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object MQTTUtils { /** @@ -36,8 +36,8 @@ object MQTTUtils { brokerUrl: String, topic: String, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[String] = { - new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) + ): ReceiverInputDStream[String] = { + new MQTTInputDStream(ssc, brokerUrl, topic, storageLevel) } /** @@ -51,7 +51,7 @@ object MQTTUtils { jssc: JavaStreamingContext, brokerUrl: String, topic: String - ): JavaDStream[String] = { + ): JavaReceiverInputDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, brokerUrl, topic) } @@ -68,7 +68,7 @@ object MQTTUtils { brokerUrl: String, topic: String, storageLevel: StorageLevel - ): JavaDStream[String] = { + ): JavaReceiverInputDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, brokerUrl, topic, storageLevel) } diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java index 44743aaecf986..ce5aa1e0cdda4 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java @@ -18,7 +18,7 @@ package org.apache.spark.streaming.mqtt; import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.junit.Test; import org.apache.spark.streaming.LocalJavaStreamingContext; @@ -30,8 +30,8 @@ public void testMQTTStream() { String topic = "def"; // tests the API, does not actually test data receiving - JavaDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); - JavaDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, + JavaReceiverInputDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); + JavaReceiverInputDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2()); } } 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 89c40ad4619c9..467fd263e2d64 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 @@ -19,6 +19,7 @@ package org.apache.spark.streaming.mqtt import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.ReceiverInputDStream class MQTTStreamSuite extends TestSuiteBase { @@ -28,8 +29,9 @@ class MQTTStreamSuite extends TestSuiteBase { val topic = "def" // tests the API, does not actually test data receiving - val test1 = MQTTUtils.createStream(ssc, brokerUrl, topic) - val test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1: ReceiverInputDStream[String] = MQTTUtils.createStream(ssc, brokerUrl, topic) + val test2: ReceiverInputDStream[String] = + MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data ssc.stop() diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 398b9f4fbaa7d..a443459594710 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -31,20 +31,6 @@ Spark Project External Twitter http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 843a4a7a9ad72..7bca1407116fa 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -25,6 +25,8 @@ import twitter4j.auth.OAuthAuthorization import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.Logging +import org.apache.spark.streaming.receiver.Receiver /* A stream of Twitter statuses, potentially filtered by one or more keywords. * @@ -41,7 +43,7 @@ class TwitterInputDStream( twitterAuth: Option[Authorization], filters: Seq[String], storageLevel: StorageLevel - ) extends NetworkInputDStream[Status](ssc_) { + ) extends ReceiverInputDStream[Status](ssc_) { private def createOAuthAuthorization(): Authorization = { new OAuthAuthorization(new ConfigurationBuilder().build()) @@ -49,7 +51,7 @@ class TwitterInputDStream( private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) - override def getReceiver(): NetworkReceiver[Status] = { + override def getReceiver(): Receiver[Status] = { new TwitterReceiver(authorization, filters, storageLevel) } } @@ -59,27 +61,27 @@ class TwitterReceiver( twitterAuth: Authorization, filters: Seq[String], storageLevel: StorageLevel - ) extends NetworkReceiver[Status] { + ) extends Receiver[Status](storageLevel) with Logging { var twitterStream: TwitterStream = _ - lazy val blockGenerator = new BlockGenerator(storageLevel) - protected override def onStart() { - blockGenerator.start() + def onStart() { twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) twitterStream.addListener(new StatusListener { def onStatus(status: Status) = { - blockGenerator += status + store(status) } // Unimplemented def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} def onTrackLimitationNotice(i: Int) {} def onScrubGeo(l: Long, l1: Long) {} def onStallWarning(stallWarning: StallWarning) {} - def onException(e: Exception) { stopOnError(e) } + def onException(e: Exception) { + restart("Error receiving tweets", e) + } }) - val query: FilterQuery = new FilterQuery + val query = new FilterQuery if (filters.size > 0) { query.track(filters.toArray) twitterStream.filter(query) @@ -89,8 +91,7 @@ class TwitterReceiver( logInfo("Twitter receiver started") } - protected override def onStop() { - blockGenerator.stop() + def onStop() { twitterStream.shutdown() logInfo("Twitter receiver stopped") } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala index e8433b7e9f6fa..c6a9a2b73714f 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala @@ -21,8 +21,8 @@ import twitter4j.Status import twitter4j.auth.Authorization import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object TwitterUtils { /** @@ -40,7 +40,7 @@ object TwitterUtils { twitterAuth: Option[Authorization], filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[Status] = { + ): ReceiverInputDStream[Status] = { new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) } @@ -52,7 +52,7 @@ object TwitterUtils { * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. * @param jssc JavaStreamingContext object */ - def createStream(jssc: JavaStreamingContext): JavaDStream[Status] = { + def createStream(jssc: JavaStreamingContext): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, None) } @@ -65,7 +65,8 @@ object TwitterUtils { * @param jssc JavaStreamingContext object * @param filters Set of filter strings to get only those tweets that match them */ - def createStream(jssc: JavaStreamingContext, filters: Array[String]): JavaDStream[Status] = { + def createStream(jssc: JavaStreamingContext, filters: Array[String] + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, None, filters) } @@ -82,7 +83,7 @@ object TwitterUtils { jssc: JavaStreamingContext, filters: Array[String], storageLevel: StorageLevel - ): JavaDStream[Status] = { + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, None, filters, storageLevel) } @@ -92,7 +93,8 @@ object TwitterUtils { * @param jssc JavaStreamingContext object * @param twitterAuth Twitter4J Authorization */ - def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization): JavaDStream[Status] = { + def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, Some(twitterAuth)) } @@ -107,7 +109,7 @@ object TwitterUtils { jssc: JavaStreamingContext, twitterAuth: Authorization, filters: Array[String] - ): JavaDStream[Status] = { + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, Some(twitterAuth), filters) } @@ -123,7 +125,7 @@ object TwitterUtils { twitterAuth: Authorization, filters: Array[String], storageLevel: StorageLevel - ): JavaDStream[Status] = { + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel) } } diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala index 06ab0cdaf3b4e..93741e0375164 100644 --- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala +++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.streaming.twitter import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel import twitter4j.auth.{NullAuthorization, Authorization} +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import twitter4j.Status class TwitterStreamSuite extends TestSuiteBase { @@ -29,13 +31,17 @@ class TwitterStreamSuite extends TestSuiteBase { val authorization: Authorization = NullAuthorization.getInstance() // tests the API, does not actually test data receiving - val test1 = TwitterUtils.createStream(ssc, None) - val test2 = TwitterUtils.createStream(ssc, None, filters) - val test3 = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) - val test4 = TwitterUtils.createStream(ssc, Some(authorization)) - val test5 = TwitterUtils.createStream(ssc, Some(authorization), filters) - val test6 = TwitterUtils.createStream(ssc, Some(authorization), filters, - StorageLevel.MEMORY_AND_DISK_SER_2) + val test1: ReceiverInputDStream[Status] = TwitterUtils.createStream(ssc, None) + val test2: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, None, filters) + val test3: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) + val test4: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, Some(authorization)) + val test5: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, Some(authorization), filters) + val test6: ReceiverInputDStream[Status] = TwitterUtils.createStream( + ssc, Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2) // Note that actually testing the data receiving is hard as authentication keys are // necessary for accessing Twitter live stream diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 77e957f404645..a40e55876e640 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -31,20 +31,6 @@ Spark Project External ZeroMQ http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala index a538c38dc4d6f..554705878ee78 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -24,7 +24,7 @@ import akka.util.ByteString import akka.zeromq._ import org.apache.spark.Logging -import org.apache.spark.streaming.receivers._ +import org.apache.spark.streaming.receiver.ActorHelper /** * A receiver to subscribe to ZeroMQ stream. @@ -32,7 +32,7 @@ import org.apache.spark.streaming.receivers._ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, subscribe: Subscribe, bytesToObjects: Seq[ByteString] => Iterator[T]) - extends Actor with Receiver with Logging { + extends Actor with ActorHelper with Logging { override def preStart() = ZeroMQExtension(context.system) .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) @@ -46,9 +46,8 @@ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, // We ignore first frame for processing as it is the topic val bytes = m.frames.tail - pushBlock(bytesToObjects(bytes)) + store(bytesToObjects(bytes)) case Closed => logInfo("received closed ") - } } diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala index b254e00714621..0469d0af8864a 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala @@ -24,10 +24,10 @@ import akka.util.ByteString import akka.zeromq.Subscribe import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream} +import org.apache.spark.streaming.receiver.ActorSupervisorStrategy object ZeroMQUtils { /** @@ -48,8 +48,8 @@ object ZeroMQUtils { subscribe: Subscribe, bytesToObjects: Seq[ByteString] => Iterator[T], storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy - ): DStream[T] = { + supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy + ): ReceiverInputDStream[T] = { ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), "ZeroMQReceiver", storageLevel, supervisorStrategy) } @@ -72,7 +72,7 @@ object ZeroMQUtils { bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], storageLevel: StorageLevel, supervisorStrategy: SupervisorStrategy - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator @@ -96,7 +96,7 @@ object ZeroMQUtils { subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], storageLevel: StorageLevel - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator @@ -119,7 +119,7 @@ object ZeroMQUtils { publisherUrl: String, subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java index d2361e14b898a..417b91eecb0ee 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming.zeromq; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.junit.Test; import akka.actor.SupervisorStrategy; import akka.util.ByteString; @@ -24,7 +25,6 @@ import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { @@ -39,11 +39,11 @@ public Iterable call(byte[][] bytes) throws Exception { } }; - JavaDStream test1 = ZeroMQUtils.createStream( + JavaReceiverInputDStream test1 = ZeroMQUtils.createStream( ssc, publishUrl, subscribe, bytesToObjects); - JavaDStream test2 = ZeroMQUtils.createStream( + JavaReceiverInputDStream test2 = ZeroMQUtils.createStream( ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test3 = ZeroMQUtils.createStream( + JavaReceiverInputDStream test3 = ZeroMQUtils.createStream( ssc,publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy()); } diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala index 92d55a7a7b6e4..cc10ff6ae03cd 100644 --- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala +++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala @@ -23,6 +23,7 @@ import akka.zeromq.Subscribe import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.streaming.dstream.ReceiverInputDStream class ZeroMQStreamSuite extends TestSuiteBase { @@ -33,10 +34,12 @@ class ZeroMQStreamSuite extends TestSuiteBase { val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]] // tests the API, does not actually test data receiving - val test1 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) - val test2 = ZeroMQUtils.createStream( + val test1: ReceiverInputDStream[String] = + ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) + val test2: ReceiverInputDStream[String] = ZeroMQUtils.createStream( ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) - val test3 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects, + val test3: ReceiverInputDStream[String] = ZeroMQUtils.createStream( + ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy) // TODO: Actually test data receiving diff --git a/graphx/pom.xml b/graphx/pom.xml index b4c67ddcd8ca9..dc108d2fe7fbd 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -31,20 +31,6 @@ Spark Project GraphX http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 45349692cbf6c..50395868902dc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -46,7 +46,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * @note vertex ids are unique. * @return an RDD containing the vertices in this graph */ - val vertices: VertexRDD[VD] + @transient val vertices: VertexRDD[VD] /** * An RDD containing the edges and their associated attributes. The entries in the RDD contain @@ -59,7 +59,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * along with their vertex data. * */ - val edges: EdgeRDD[ED] + @transient val edges: EdgeRDD[ED] /** * An RDD containing the edge triplets, which are edges along with the vertex data associated with @@ -77,7 +77,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * val numInvalid = graph.triplets.map(e => if (e.src.data == e.dst.data) 1 else 0).sum * }}} */ - val triplets: RDD[EdgeTriplet[VD, ED]] + @transient val triplets: RDD[EdgeTriplet[VD, ED]] /** * Caches the vertices and edges associated with this graph at the specified storage level. 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 5635287694ee2..4997fbc3cbcd8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -34,28 +34,28 @@ import scala.util.Random class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Serializable { /** The number of edges in the graph. */ - lazy val numEdges: Long = graph.edges.count() + @transient lazy val numEdges: Long = graph.edges.count() /** The number of vertices in the graph. */ - lazy val numVertices: Long = graph.vertices.count() + @transient lazy val numVertices: Long = graph.vertices.count() /** * The in-degree of each vertex in the graph. * @note Vertices with no in-edges are not returned in the resulting RDD. */ - lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) + @transient lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) /** * The out-degree of each vertex in the graph. * @note Vertices with no out-edges are not returned in the resulting RDD. */ - lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) + @transient lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) /** * The degree of each vertex in the graph. * @note Vertices with no edges are not returned in the resulting RDD. */ - lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) + @transient lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) /** * Computes the neighboring vertex degrees. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 2e05f5d4e4969..b7c472e905a9b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -34,10 +34,10 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap */ private[graphx] class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( - val srcIds: Array[VertexId], - val dstIds: Array[VertexId], - val data: Array[ED], - val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable { + @transient val srcIds: Array[VertexId], + @transient val dstIds: Array[VertexId], + @transient val data: Array[ED], + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable { /** * Reverse all the edges in this partition. 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 a467ca1ae715a..ea94d4accb63b 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -165,8 +165,11 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { // not have any edges in the specified direction. assert(edges.count === 50) edges.collect.foreach { - case (vid, edges) => if (vid > 0 && vid < 49) assert(edges.size == 2) - else assert(edges.size == 1) + case (vid, edges) => if (vid > 0 && vid < 49) { + assert(edges.size == 2) + } else { + assert(edges.size == 1) + } } edges.collect.foreach { case (vid, edges) => diff --git a/make-distribution.sh b/make-distribution.sh index 5c780fcbda863..c05dcd89d90a7 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -28,38 +28,35 @@ # --tgz: Additionally creates spark-$VERSION-bin.tar.gz # --hadoop VERSION: Builds against specified version of Hadoop. # --with-yarn: Enables support for Hadoop YARN. +# --with-hive: Enable support for reading Hive tables. +# --name: A moniker for the release target. Defaults to the Hadoop verison. # # Recommended deploy/testing procedure (standalone mode): # 1) Rsync / deploy the dist/ dir to one host # 2) cd to deploy dir; ./sbin/start-master.sh # 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. # 4) ./sbin/start-slave.sh 1 <> -# 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell +# 5) ./bin/spark-shell --master spark://my-master-ip:7077 # # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -# Get version from SBT -export TERM=dumb # Prevents color codes in SBT output - -VERSIONSTRING=$($FWDIR/sbt/sbt "show version") - +VERSION=$(mvn help:evaluate -Dexpression=project.version | grep -v "INFO" | tail -n 1) if [ $? == -1 ] ;then - echo -e "You need sbt installed and available on your path." - echo -e "Download sbt from http://www.scala-sbt.org/" + echo -e "You need Maven installed to build Spark." + echo -e "Download Maven from https://maven.apache.org." exit -1; fi -VERSION=$(echo "${VERSIONSTRING}" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') -echo "Version is ${VERSION}" - # Initialize defaults SPARK_HADOOP_VERSION=1.0.4 SPARK_YARN=false +SPARK_HIVE=false SPARK_TACHYON=false MAKE_TGZ=false +NAME=none # Parse arguments while (( "$#" )); do @@ -71,23 +68,37 @@ while (( "$#" )); do --with-yarn) SPARK_YARN=true ;; + --with-hive) + SPARK_HIVE=true + ;; --with-tachyon) SPARK_TACHYON=true ;; --tgz) MAKE_TGZ=true ;; + --name) + NAME="$2" + shift + ;; esac shift done +if [ "$NAME" == "none" ]; then + NAME=$SPARK_HADOOP_VERSION +fi + +echo "Spark version is $VERSION" + if [ "$MAKE_TGZ" == "true" ]; then - echo "Making spark-$VERSION-hadoop_$SPARK_HADOOP_VERSION-bin.tar.gz" + echo "Making spark-$VERSION-bin-$NAME.tgz" else - echo "Making distribution for Spark $VERSION in $DISTDIR..." + echo "Making distribution for Spark $VERSION in $DISTDIR..." fi echo "Hadoop version set to $SPARK_HADOOP_VERSION" +echo "Release name set to $NAME" if [ "$SPARK_YARN" == "true" ]; then echo "YARN enabled" else @@ -100,24 +111,46 @@ else echo "Tachyon Disabled" fi -# Build fat JAR -export SPARK_HADOOP_VERSION -export SPARK_YARN +# Build uber fat JAR cd $FWDIR -"sbt/sbt" "assembly/assembly" +export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" + +if [ "$SPARK_HIVE" == "true" ]; then + MAYBE_HIVE="-Phive" +else + MAYBE_HIVE="" +fi + +if [ "$SPARK_YARN" == "true" ]; then + if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then + mvn clean package -DskipTests -Pyarn-alpha -Dhadoop.version=$SPARK_HADOOP_VERSION \ + -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE -Phadoop-0.23 + else + mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \ + -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + fi +else + if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then + mvn clean package -Phadoop-0.23 -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + else + mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + fi +fi # Make directories rm -rf "$DISTDIR" -mkdir -p "$DISTDIR/jars" +mkdir -p "$DISTDIR/lib" echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" # Copy jars -cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/jars/" +cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" +cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf +cp "$FWDIR"/conf/slaves "$DISTDIR"/conf cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" @@ -135,16 +168,16 @@ if [ "$SPARK_TACHYON" == "true" ]; then wget "$TACHYON_URL" tar xf "tachyon-${TACHYON_VERSION}-bin.tar.gz" - cp "tachyon-${TACHYON_VERSION}/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/jars" + cp "tachyon-${TACHYON_VERSION}/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/lib" mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web" cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon" cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" if [[ `uname -a` == Darwin* ]]; then # need to run sed differently on osx - nl=$'\n'; sed -i "" -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\\$nl export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" + nl=$'\n'; sed -i "" -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\\$nl export TACHYON_JAR=\$TACHYON_HOME/../lib/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" else - sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" + sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../lib/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" fi popd > /dev/null @@ -152,8 +185,10 @@ if [ "$SPARK_TACHYON" == "true" ]; then fi if [ "$MAKE_TGZ" == "true" ]; then - TARDIR="$FWDIR/spark-$VERSION" + TARDIR_NAME=spark-$VERSION-bin-$NAME + TARDIR="$FWDIR/$TARDIR_NAME" + rm -rf "$TARDIR" cp -r "$DISTDIR" "$TARDIR" - tar -zcf "spark-$VERSION-hadoop_$SPARK_HADOOP_VERSION-bin.tar.gz" -C "$FWDIR" "spark-$VERSION" + tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$FWDIR" "$TARDIR_NAME" rm -rf "$TARDIR" fi diff --git a/mllib/data/sample_libsvm_data.txt b/mllib/data/sample_libsvm_data.txt new file mode 100644 index 0000000000000..861c70cde7fd2 --- /dev/null +++ b/mllib/data/sample_libsvm_data.txt @@ -0,0 +1,100 @@ +0 128:51 129:159 130:253 131:159 132:50 155:48 156:238 157:252 158:252 159:252 160:237 182:54 183:227 184:253 185:252 186:239 187:233 188:252 189:57 190:6 208:10 209:60 210:224 211:252 212:253 213:252 214:202 215:84 216:252 217:253 218:122 236:163 237:252 238:252 239:252 240:253 241:252 242:252 243:96 244:189 245:253 246:167 263:51 264:238 265:253 266:253 267:190 268:114 269:253 270:228 271:47 272:79 273:255 274:168 290:48 291:238 292:252 293:252 294:179 295:12 296:75 297:121 298:21 301:253 302:243 303:50 317:38 318:165 319:253 320:233 321:208 322:84 329:253 330:252 331:165 344:7 345:178 346:252 347:240 348:71 349:19 350:28 357:253 358:252 359:195 372:57 373:252 374:252 375:63 385:253 386:252 387:195 400:198 401:253 402:190 413:255 414:253 415:196 427:76 428:246 429:252 430:112 441:253 442:252 443:148 455:85 456:252 457:230 458:25 467:7 468:135 469:253 470:186 471:12 483:85 484:252 485:223 494:7 495:131 496:252 497:225 498:71 511:85 512:252 513:145 521:48 522:165 523:252 524:173 539:86 540:253 541:225 548:114 549:238 550:253 551:162 567:85 568:252 569:249 570:146 571:48 572:29 573:85 574:178 575:225 576:253 577:223 578:167 579:56 595:85 596:252 597:252 598:252 599:229 600:215 601:252 602:252 603:252 604:196 605:130 623:28 624:199 625:252 626:252 627:253 628:252 629:252 630:233 631:145 652:25 653:128 654:252 655:253 656:252 657:141 658:37 +1 159:124 160:253 161:255 162:63 186:96 187:244 188:251 189:253 190:62 214:127 215:251 216:251 217:253 218:62 241:68 242:236 243:251 244:211 245:31 246:8 268:60 269:228 270:251 271:251 272:94 296:155 297:253 298:253 299:189 323:20 324:253 325:251 326:235 327:66 350:32 351:205 352:253 353:251 354:126 378:104 379:251 380:253 381:184 382:15 405:80 406:240 407:251 408:193 409:23 432:32 433:253 434:253 435:253 436:159 460:151 461:251 462:251 463:251 464:39 487:48 488:221 489:251 490:251 491:172 515:234 516:251 517:251 518:196 519:12 543:253 544:251 545:251 546:89 570:159 571:255 572:253 573:253 574:31 597:48 598:228 599:253 600:247 601:140 602:8 625:64 626:251 627:253 628:220 653:64 654:251 655:253 656:220 681:24 682:193 683:253 684:220 +1 125:145 126:255 127:211 128:31 152:32 153:237 154:253 155:252 156:71 180:11 181:175 182:253 183:252 184:71 209:144 210:253 211:252 212:71 236:16 237:191 238:253 239:252 240:71 264:26 265:221 266:253 267:252 268:124 269:31 293:125 294:253 295:252 296:252 297:108 322:253 323:252 324:252 325:108 350:255 351:253 352:253 353:108 378:253 379:252 380:252 381:108 406:253 407:252 408:252 409:108 434:253 435:252 436:252 437:108 462:255 463:253 464:253 465:170 490:253 491:252 492:252 493:252 494:42 518:149 519:252 520:252 521:252 522:144 546:109 547:252 548:252 549:252 550:144 575:218 576:253 577:253 578:255 579:35 603:175 604:252 605:252 606:253 607:35 631:73 632:252 633:252 634:253 635:35 659:31 660:211 661:252 662:253 663:35 +1 153:5 154:63 155:197 181:20 182:254 183:230 184:24 209:20 210:254 211:254 212:48 237:20 238:254 239:255 240:48 265:20 266:254 267:254 268:57 293:20 294:254 295:254 296:108 321:16 322:239 323:254 324:143 350:178 351:254 352:143 378:178 379:254 380:143 406:178 407:254 408:162 434:178 435:254 436:240 462:113 463:254 464:240 490:83 491:254 492:245 493:31 518:79 519:254 520:246 521:38 547:214 548:254 549:150 575:144 576:241 577:8 603:144 604:240 605:2 631:144 632:254 633:82 659:230 660:247 661:40 687:168 688:209 689:31 +1 152:1 153:168 154:242 155:28 180:10 181:228 182:254 183:100 209:190 210:254 211:122 237:83 238:254 239:162 265:29 266:254 267:248 268:25 293:29 294:255 295:254 296:103 321:29 322:254 323:254 324:109 349:29 350:254 351:254 352:109 377:29 378:254 379:254 380:109 405:29 406:255 407:254 408:109 433:29 434:254 435:254 436:109 461:29 462:254 463:254 464:63 489:29 490:254 491:254 492:28 517:29 518:254 519:254 520:28 545:29 546:254 547:254 548:35 573:29 574:254 575:254 576:109 601:6 602:212 603:254 604:109 630:203 631:254 632:178 658:155 659:254 660:190 686:32 687:199 688:104 +0 130:64 131:253 132:255 133:63 157:96 158:205 159:251 160:253 161:205 162:111 163:4 184:96 185:189 186:251 187:251 188:253 189:251 190:251 191:31 209:16 210:64 211:223 212:244 213:251 214:251 215:211 216:213 217:251 218:251 219:31 236:80 237:181 238:251 239:253 240:251 241:251 242:251 243:94 244:96 245:251 246:251 247:31 263:92 264:253 265:253 266:253 267:255 268:253 269:253 270:253 271:95 272:96 273:253 274:253 275:31 290:92 291:236 292:251 293:243 294:220 295:233 296:251 297:251 298:243 299:82 300:96 301:251 302:251 303:31 317:80 318:253 319:251 320:251 321:188 323:96 324:251 325:251 326:109 328:96 329:251 330:251 331:31 344:96 345:240 346:253 347:243 348:188 349:42 351:96 352:204 353:109 354:4 356:12 357:197 358:251 359:31 372:221 373:251 374:253 375:121 379:36 380:23 385:190 386:251 387:31 399:48 400:234 401:253 413:191 414:253 415:31 426:44 427:221 428:251 429:251 440:12 441:197 442:251 443:31 454:190 455:251 456:251 457:251 468:96 469:251 470:251 471:31 482:190 483:251 484:251 485:113 495:40 496:234 497:251 498:219 499:23 510:190 511:251 512:251 513:94 522:40 523:217 524:253 525:231 526:47 538:191 539:253 540:253 541:253 548:12 549:174 550:253 551:253 552:219 553:39 566:67 567:236 568:251 569:251 570:191 571:190 572:111 573:72 574:190 575:191 576:197 577:251 578:243 579:121 580:39 595:63 596:236 597:251 598:253 599:251 600:251 601:251 602:251 603:253 604:251 605:188 606:94 624:27 625:129 626:253 627:251 628:251 629:251 630:251 631:229 632:168 633:15 654:95 655:212 656:251 657:211 658:94 659:59 +1 159:121 160:254 161:136 186:13 187:230 188:253 189:248 190:99 213:4 214:118 215:253 216:253 217:225 218:42 241:61 242:253 243:253 244:253 245:74 268:32 269:206 270:253 271:253 272:186 273:9 296:211 297:253 298:253 299:239 300:69 324:254 325:253 326:253 327:133 351:142 352:255 353:253 354:186 355:8 378:149 379:229 380:254 381:207 382:21 405:54 406:229 407:253 408:254 409:105 433:152 434:254 435:254 436:213 437:26 460:112 461:251 462:253 463:253 464:26 487:29 488:212 489:253 490:250 491:149 514:36 515:214 516:253 517:253 518:137 542:75 543:253 544:253 545:253 546:59 570:93 571:253 572:253 573:189 574:17 598:224 599:253 600:253 601:84 625:43 626:235 627:253 628:126 629:1 653:99 654:248 655:253 656:119 682:225 683:235 684:49 +1 100:166 101:222 102:55 128:197 129:254 130:218 131:5 155:29 156:249 157:254 158:254 159:9 183:45 184:254 185:254 186:174 187:2 210:4 211:164 212:254 213:254 214:85 238:146 239:254 240:254 241:254 242:85 265:101 266:245 267:254 268:254 269:254 270:85 292:97 293:248 294:254 295:204 296:254 297:254 298:85 315:12 316:59 317:98 318:151 319:237 320:254 321:254 322:109 323:35 324:254 325:254 326:85 343:41 344:216 345:254 346:254 347:239 348:153 349:37 350:4 351:32 352:254 353:254 354:85 372:7 373:44 374:44 375:30 379:32 380:254 381:254 382:96 407:19 408:230 409:254 410:174 436:197 437:254 438:110 464:197 465:254 466:85 492:197 493:253 494:63 515:37 516:54 517:54 518:45 519:26 520:84 521:221 522:84 523:21 524:31 525:162 526:78 540:6 541:41 542:141 543:244 544:254 545:254 546:248 547:236 548:254 549:254 550:254 551:233 552:239 553:254 554:138 567:23 568:167 569:254 570:254 571:254 572:254 573:229 574:228 575:185 576:138 577:138 578:138 579:138 580:138 581:138 582:44 595:113 596:254 597:254 598:254 599:179 600:64 601:5 623:32 624:209 625:183 626:97 +0 155:53 156:255 157:253 158:253 159:253 160:124 183:180 184:253 185:251 186:251 187:251 188:251 189:145 190:62 209:32 210:217 211:241 212:253 213:251 214:251 215:251 216:251 217:253 218:107 237:37 238:251 239:251 240:253 241:251 242:251 243:251 244:251 245:253 246:107 265:166 266:251 267:251 268:253 269:251 270:96 271:148 272:251 273:253 274:107 291:73 292:253 293:253 294:253 295:253 296:130 299:110 300:253 301:255 302:108 319:73 320:251 321:251 322:251 323:251 327:109 328:251 329:253 330:107 347:202 348:251 349:251 350:251 351:225 354:6 355:129 356:251 357:253 358:107 375:150 376:251 377:251 378:251 379:71 382:115 383:251 384:251 385:253 386:107 403:253 404:251 405:251 406:173 407:20 410:217 411:251 412:251 413:253 414:107 430:182 431:255 432:253 433:216 438:218 439:253 440:253 441:182 457:63 458:221 459:253 460:251 461:215 465:84 466:236 467:251 468:251 469:77 485:109 486:251 487:253 488:251 489:215 492:11 493:160 494:251 495:251 496:96 513:109 514:251 515:253 516:251 517:137 520:150 521:251 522:251 523:251 524:71 541:109 542:251 543:253 544:251 545:35 547:130 548:253 549:251 550:251 551:173 552:20 569:110 570:253 571:255 572:253 573:98 574:150 575:253 576:255 577:253 578:164 597:109 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:35 625:93 626:241 627:253 628:251 629:251 630:251 631:251 632:216 633:112 634:5 654:103 655:253 656:251 657:251 658:251 659:251 683:124 684:251 685:225 686:71 687:71 +0 128:73 129:253 130:227 131:73 132:21 156:73 157:251 158:251 159:251 160:174 182:16 183:166 184:228 185:251 186:251 187:251 188:122 210:62 211:220 212:253 213:251 214:251 215:251 216:251 217:79 238:79 239:231 240:253 241:251 242:251 243:251 244:251 245:232 246:77 264:145 265:253 266:253 267:253 268:255 269:253 270:253 271:253 272:253 273:255 274:108 292:144 293:251 294:251 295:251 296:253 297:168 298:107 299:169 300:251 301:253 302:189 303:20 318:27 319:89 320:236 321:251 322:235 323:215 324:164 325:15 326:6 327:129 328:251 329:253 330:251 331:35 345:47 346:211 347:253 348:251 349:251 350:142 354:37 355:251 356:251 357:253 358:251 359:35 373:109 374:251 375:253 376:251 377:251 378:142 382:11 383:148 384:251 385:253 386:251 387:164 400:11 401:150 402:253 403:255 404:211 405:25 410:11 411:150 412:253 413:255 414:211 415:25 428:140 429:251 430:251 431:253 432:107 438:37 439:251 440:251 441:211 442:46 456:190 457:251 458:251 459:253 460:128 461:5 466:37 467:251 468:251 469:51 484:115 485:251 486:251 487:253 488:188 489:20 492:32 493:109 494:129 495:251 496:173 497:103 512:217 513:251 514:251 515:201 516:30 520:73 521:251 522:251 523:251 524:71 540:166 541:253 542:253 543:255 544:149 545:73 546:150 547:253 548:255 549:253 550:253 551:143 568:140 569:251 570:251 571:253 572:251 573:251 574:251 575:251 576:253 577:251 578:230 579:61 596:190 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:242 605:215 606:55 624:21 625:189 626:251 627:253 628:251 629:251 630:251 631:173 632:103 653:31 654:200 655:253 656:251 657:96 658:71 659:20 +1 155:178 156:255 157:105 182:6 183:188 184:253 185:216 186:14 210:14 211:202 212:253 213:253 214:23 238:12 239:199 240:253 241:128 242:6 266:42 267:253 268:253 269:158 294:42 295:253 296:253 297:158 322:155 323:253 324:253 325:158 350:160 351:253 352:253 353:147 378:160 379:253 380:253 381:41 405:17 406:225 407:253 408:235 409:31 433:24 434:253 435:253 436:176 461:24 462:253 463:253 464:176 489:24 490:253 491:253 492:176 517:24 518:253 519:253 520:176 545:24 546:253 547:253 548:162 573:46 574:253 575:253 576:59 601:142 602:253 603:253 604:59 629:142 630:253 631:253 632:59 657:142 658:253 659:202 660:8 685:87 686:253 687:139 +0 154:46 155:105 156:254 157:254 158:254 159:254 160:255 161:239 162:41 180:37 181:118 182:222 183:254 184:253 185:253 186:253 187:253 188:253 189:253 190:211 191:54 207:14 208:200 209:253 210:253 211:254 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:116 233:16 234:160 235:236 236:253 237:253 238:253 239:254 240:253 241:253 242:246 243:229 244:253 245:253 246:253 247:116 261:99 262:253 263:253 264:253 265:253 266:253 267:254 268:253 269:253 270:213 271:99 272:253 273:253 274:253 275:116 288:25 289:194 290:253 291:253 292:253 293:253 294:131 295:97 296:169 297:253 298:93 299:99 300:253 301:253 302:253 303:116 316:206 317:253 318:253 319:251 320:233 321:127 322:9 324:18 325:38 326:3 327:15 328:171 329:253 330:253 331:116 343:55 344:240 345:253 346:253 347:233 355:31 356:186 357:253 358:253 359:116 371:176 372:253 373:253 374:253 375:127 383:99 384:253 385:253 386:253 387:116 399:176 400:253 401:253 402:131 403:9 411:99 412:253 413:253 414:253 415:116 426:119 427:254 428:254 429:232 430:75 440:158 441:254 442:254 443:117 454:118 455:253 456:253 457:154 468:156 469:253 470:253 471:116 482:118 483:253 484:253 485:154 496:156 497:253 498:253 499:116 509:46 510:222 511:253 512:253 513:154 522:7 523:116 524:246 525:253 526:180 527:9 538:118 539:253 540:253 541:154 550:116 551:253 552:253 553:253 554:174 566:118 567:253 568:253 569:154 577:110 578:246 579:253 580:253 581:240 582:67 594:118 595:253 596:253 597:238 598:215 599:49 600:20 601:20 602:20 603:66 604:215 605:241 606:253 607:245 608:233 609:64 622:82 623:229 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:254 632:253 633:253 634:240 635:107 651:176 652:253 653:253 654:253 655:253 656:253 657:253 658:253 659:254 660:253 661:253 662:108 679:40 680:239 681:253 682:253 683:253 684:253 685:253 686:253 687:254 688:161 689:57 690:4 +0 152:56 153:105 154:220 155:254 156:63 178:18 179:166 180:233 181:253 182:253 183:253 184:236 185:209 186:209 187:209 188:77 189:18 206:84 207:253 208:253 209:253 210:253 211:253 212:254 213:253 214:253 215:253 216:253 217:172 218:8 233:57 234:238 235:253 236:253 237:253 238:253 239:253 240:254 241:253 242:253 243:253 244:253 245:253 246:119 260:14 261:238 262:253 263:253 264:253 265:253 266:253 267:253 268:179 269:196 270:253 271:253 272:253 273:253 274:238 275:12 288:33 289:253 290:253 291:253 292:253 293:253 294:248 295:134 297:18 298:83 299:237 300:253 301:253 302:253 303:14 316:164 317:253 318:253 319:253 320:253 321:253 322:128 327:57 328:119 329:214 330:253 331:94 343:57 344:248 345:253 346:253 347:253 348:126 349:14 350:4 357:179 358:253 359:248 360:56 371:175 372:253 373:253 374:240 375:190 376:28 385:179 386:253 387:253 388:173 399:209 400:253 401:253 402:178 413:92 414:253 415:253 416:208 427:211 428:254 429:254 430:179 442:135 443:255 444:209 455:209 456:253 457:253 458:90 470:134 471:253 472:208 483:209 484:253 485:253 486:178 497:2 498:142 499:253 500:208 511:209 512:253 513:253 514:214 515:35 525:30 526:253 527:253 528:208 539:165 540:253 541:253 542:253 543:215 544:36 553:163 554:253 555:253 556:164 567:18 568:172 569:253 570:253 571:253 572:214 573:127 574:7 580:72 581:232 582:253 583:171 584:17 596:8 597:182 598:253 599:253 600:253 601:253 602:162 603:56 607:64 608:240 609:253 610:253 611:14 625:7 626:173 627:253 628:253 629:253 630:253 631:245 632:241 633:239 634:239 635:246 636:253 637:225 638:14 639:1 654:18 655:59 656:138 657:224 658:253 659:253 660:254 661:253 662:253 663:253 664:240 665:96 685:37 686:104 687:192 688:255 689:253 690:253 691:182 692:73 +1 130:7 131:176 132:254 133:224 158:51 159:253 160:253 161:223 185:4 186:170 187:253 188:253 189:214 213:131 214:253 215:253 216:217 217:39 241:209 242:253 243:253 244:134 268:75 269:240 270:253 271:239 272:26 296:184 297:253 298:245 299:63 323:142 324:255 325:253 326:185 350:62 351:229 352:254 353:242 354:73 377:54 378:229 379:253 380:254 381:105 405:152 406:254 407:254 408:213 409:26 432:32 433:243 434:253 435:253 436:115 459:2 460:142 461:253 462:253 463:155 487:30 488:253 489:253 490:232 491:55 515:75 516:253 517:253 518:164 542:72 543:232 544:253 545:189 546:17 570:224 571:253 572:253 573:163 597:43 598:235 599:253 600:253 601:195 602:21 625:28 626:231 627:253 628:253 629:184 630:14 654:225 655:253 656:253 657:75 +0 155:21 156:176 157:253 158:253 159:124 182:105 183:176 184:251 185:251 186:251 187:251 188:105 208:58 209:217 210:241 211:253 212:251 213:251 214:251 215:251 216:243 217:113 218:5 235:63 236:231 237:251 238:251 239:253 240:251 241:251 242:251 243:251 244:253 245:251 246:113 263:144 264:251 265:251 266:251 267:253 268:251 269:251 270:251 271:251 272:253 273:251 274:215 290:125 291:253 292:253 293:253 294:253 295:255 296:253 297:253 298:253 299:253 300:255 301:253 302:227 303:42 318:253 319:251 320:251 321:251 322:251 323:253 324:251 325:251 326:251 327:251 328:253 329:251 330:251 331:142 345:27 346:253 347:251 348:251 349:235 350:241 351:253 352:251 353:246 354:137 355:35 356:98 357:251 358:251 359:236 360:61 372:47 373:211 374:253 375:251 376:235 377:82 378:103 379:253 380:251 381:137 384:73 385:251 386:251 387:251 388:71 399:27 400:211 401:251 402:253 403:251 404:86 407:72 408:71 409:10 412:73 413:251 414:251 415:173 416:20 427:89 428:253 429:253 430:255 431:253 432:35 440:73 441:253 442:253 443:253 444:72 454:84 455:236 456:251 457:251 458:253 459:251 460:138 468:73 469:251 470:251 471:251 472:71 481:63 482:236 483:251 484:251 485:251 486:227 487:251 488:246 489:138 490:11 494:16 495:37 496:228 497:251 498:246 499:137 500:10 509:73 510:251 511:251 512:251 513:173 514:42 515:142 516:142 517:142 518:41 522:109 523:251 524:253 525:251 526:137 537:73 538:251 539:251 540:173 541:20 549:27 550:211 551:251 552:253 553:147 554:10 565:73 566:253 567:253 568:143 575:21 576:176 577:253 578:253 579:253 593:73 594:251 595:251 596:205 597:144 603:176 604:251 605:251 606:188 607:107 621:62 622:236 623:251 624:251 625:251 626:218 627:217 628:217 629:217 630:217 631:253 632:230 633:189 634:20 650:83 651:158 652:251 653:251 654:253 655:251 656:251 657:251 658:251 659:253 660:107 679:37 680:251 681:251 682:253 683:251 684:251 685:251 686:122 687:72 688:30 +1 151:68 152:45 153:131 154:131 155:131 156:101 157:68 158:92 159:44 187:19 188:170 211:29 212:112 213:89 215:40 216:222 239:120 240:254 241:251 242:127 243:40 244:222 267:197 268:254 269:254 270:91 271:40 272:222 294:64 295:247 296:254 297:236 298:50 299:40 300:107 322:184 323:254 324:254 325:91 327:6 328:14 350:203 351:254 352:254 353:71 377:23 378:218 379:254 380:254 381:71 405:113 406:254 407:255 408:239 409:53 433:210 434:254 435:254 436:195 460:62 461:242 462:254 463:241 464:88 468:28 488:86 489:254 490:254 491:189 495:28 496:104 516:106 517:254 518:254 519:168 523:40 524:91 544:216 545:254 546:245 547:51 551:35 552:80 572:216 573:254 574:102 599:55 600:239 601:254 602:52 627:166 628:254 629:210 630:23 655:223 656:252 657:104 683:223 684:169 +0 125:29 126:170 127:255 128:255 129:141 151:29 152:198 153:255 154:255 155:255 156:226 157:255 158:86 178:141 179:255 180:255 181:170 182:29 184:86 185:255 186:255 187:141 204:29 205:226 206:255 207:198 208:57 213:226 214:255 215:255 216:226 217:114 231:29 232:255 233:255 234:114 241:141 242:170 243:114 244:255 245:255 246:141 259:226 260:255 261:170 269:29 270:57 273:141 274:255 275:226 286:57 287:255 288:170 302:114 303:255 304:198 314:226 315:255 331:170 332:255 333:57 342:255 343:226 360:255 361:170 370:255 371:170 388:114 389:198 398:255 399:226 416:86 417:255 426:198 427:255 444:86 445:255 454:114 455:255 456:57 472:86 473:255 482:29 483:255 484:226 500:141 501:255 511:170 512:255 513:170 528:226 529:198 539:29 540:226 541:255 542:170 555:29 556:255 557:114 568:29 569:226 570:255 571:141 582:57 583:226 584:226 598:141 599:255 600:255 601:170 602:86 607:29 608:86 609:226 610:255 611:226 612:29 627:86 628:198 629:255 630:255 631:255 632:255 633:255 634:255 635:255 636:255 637:255 638:141 639:29 657:29 658:114 659:170 660:170 661:170 662:170 663:170 664:86 +0 153:203 154:254 155:252 156:252 157:252 158:214 159:51 160:20 180:62 181:221 182:252 183:250 184:250 185:250 186:252 187:250 188:160 189:20 207:62 208:211 209:250 210:252 211:250 212:250 213:250 214:252 215:250 216:250 217:49 234:41 235:221 236:250 237:250 238:252 239:250 240:250 241:250 242:252 243:250 244:128 245:10 262:254 263:252 264:252 265:252 266:254 267:252 268:252 269:252 270:254 271:252 272:252 273:90 290:150 291:190 292:250 293:250 294:252 295:250 296:250 297:169 298:171 299:250 300:250 301:250 302:82 318:31 319:191 320:250 321:250 322:252 323:189 324:100 325:20 326:172 327:250 328:250 329:250 330:80 346:213 347:250 348:250 349:250 350:212 351:29 354:252 355:250 356:250 357:250 374:92 375:252 376:252 377:252 382:51 383:252 384:252 385:252 386:203 401:82 402:252 403:250 404:250 405:169 410:132 411:250 412:250 413:250 414:121 428:92 429:231 430:252 431:250 432:159 433:20 438:252 439:250 440:250 441:250 456:30 457:211 458:252 459:250 460:221 461:40 466:90 467:250 468:250 469:250 470:163 484:31 485:213 486:254 487:232 488:80 494:92 495:252 496:252 497:212 498:163 512:151 513:250 514:252 515:149 522:252 523:250 524:250 525:49 540:60 541:221 542:252 543:210 544:60 550:252 551:250 552:250 553:49 569:202 570:252 571:250 572:221 573:40 576:123 577:202 578:252 579:250 580:250 581:49 596:123 597:243 598:255 599:252 600:252 601:252 602:254 603:252 604:252 605:252 606:254 607:252 608:100 625:121 626:171 627:250 628:250 629:250 630:252 631:250 632:250 633:250 634:252 635:250 636:100 654:20 655:160 656:250 657:250 658:252 659:250 660:250 661:250 662:252 663:189 664:40 683:20 684:170 685:250 686:252 687:250 688:128 689:49 690:49 691:29 +1 98:64 99:191 100:70 125:68 126:243 127:253 128:249 129:63 152:30 153:223 154:253 155:253 156:247 157:41 179:73 180:238 181:253 182:253 183:253 184:242 206:73 207:236 208:253 209:253 210:253 211:253 212:242 234:182 235:253 236:253 237:191 238:247 239:253 240:149 262:141 263:253 264:143 265:86 266:249 267:253 268:122 290:9 291:36 292:7 293:14 294:233 295:253 296:122 322:230 323:253 324:122 350:230 351:253 352:122 378:231 379:255 380:123 406:230 407:253 408:52 433:61 434:245 435:253 461:98 462:253 463:253 468:35 469:12 489:98 490:253 491:253 494:9 495:142 496:233 497:146 517:190 518:253 519:253 520:128 521:7 522:99 523:253 524:253 525:180 544:29 545:230 546:253 547:253 548:252 549:210 550:253 551:253 552:253 553:140 571:28 572:207 573:253 574:253 575:253 576:254 577:253 578:253 579:235 580:70 581:9 599:126 600:253 601:253 602:253 603:253 604:254 605:253 606:168 607:19 627:79 628:253 629:253 630:201 631:190 632:132 633:63 634:5 +1 125:26 126:240 127:72 153:25 154:238 155:208 182:209 183:226 184:14 210:209 211:254 212:43 238:175 239:254 240:128 266:63 267:254 268:204 294:107 295:254 296:204 322:88 323:254 324:204 350:55 351:254 352:204 378:126 379:254 380:204 406:126 407:254 408:189 434:169 435:254 436:121 462:209 463:254 464:193 490:209 491:254 492:111 517:22 518:235 519:254 520:37 545:137 546:254 547:227 548:16 573:205 574:255 575:185 601:205 602:254 603:125 629:205 630:254 631:125 657:111 658:212 659:43 +0 155:62 156:91 157:213 158:255 159:228 160:91 161:12 182:70 183:230 184:253 185:253 186:253 187:253 188:253 189:152 190:7 210:246 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:106 237:21 238:247 239:253 240:253 241:253 242:253 243:253 244:253 245:208 246:24 265:156 266:253 267:253 268:253 269:253 270:253 271:253 272:253 273:195 292:88 293:238 294:253 295:253 296:253 297:221 298:253 299:253 300:253 301:195 320:230 321:253 322:253 323:253 324:198 325:40 326:177 327:253 328:253 329:195 346:56 347:156 348:251 349:253 350:189 351:182 352:15 354:86 355:240 356:253 357:210 358:28 374:213 375:253 376:253 377:156 378:3 383:205 384:253 385:253 386:106 401:121 402:252 403:253 404:135 405:3 411:46 412:253 413:253 414:106 428:28 429:212 430:253 431:248 432:23 439:42 440:253 441:253 442:106 456:197 457:253 458:234 459:70 467:42 468:253 469:253 470:106 483:11 484:202 485:253 486:187 495:58 496:253 497:210 498:27 511:107 512:253 513:253 514:40 522:53 523:227 524:253 525:195 539:107 540:253 541:253 542:40 549:47 550:227 551:253 552:231 553:58 567:107 568:253 569:253 570:40 575:5 576:131 577:222 578:253 579:231 580:59 595:14 596:204 597:253 598:226 599:222 600:73 601:58 602:58 603:170 604:253 605:253 606:227 607:58 624:197 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:238 634:58 652:33 653:179 654:241 655:253 656:253 657:253 658:253 659:250 660:116 661:14 682:75 683:179 684:253 685:151 686:89 687:86 +1 157:42 158:228 159:253 160:253 185:144 186:251 187:251 188:251 212:89 213:236 214:251 215:235 216:215 239:79 240:253 241:251 242:251 243:142 267:180 268:253 269:251 270:251 271:142 294:32 295:202 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:112 349:6 350:129 351:251 352:253 353:127 354:5 377:37 378:251 379:251 380:253 381:107 405:166 406:251 407:251 408:201 409:30 432:42 433:228 434:253 435:253 460:144 461:251 462:251 463:147 487:63 488:236 489:251 490:251 491:71 515:150 516:251 517:251 518:204 519:41 543:253 544:251 545:251 546:142 571:255 572:253 573:164 598:105 599:253 600:251 601:35 626:180 627:253 628:251 629:35 654:180 655:253 656:251 657:35 682:180 683:253 684:251 685:35 +1 128:62 129:254 130:213 156:102 157:253 158:252 159:102 160:20 184:102 185:254 186:253 187:254 188:50 212:102 213:253 214:252 215:253 216:50 240:102 241:254 242:253 243:254 244:50 268:142 269:253 270:252 271:253 272:50 295:51 296:253 297:254 298:253 299:224 300:20 323:132 324:252 325:253 326:252 327:162 351:173 352:253 353:254 354:253 355:102 378:82 379:253 380:252 381:253 382:252 383:61 406:203 407:254 408:253 409:254 410:233 433:41 434:243 435:253 436:252 437:253 438:111 461:132 462:253 463:254 464:253 465:203 488:41 489:253 490:252 491:253 492:252 493:40 515:11 516:213 517:254 518:253 519:254 520:151 543:92 544:252 545:253 546:252 547:192 548:50 570:21 571:214 572:253 573:255 574:253 575:41 598:142 599:253 600:252 601:253 602:171 625:113 626:253 627:255 628:253 629:203 630:40 653:30 654:131 655:233 656:111 +0 154:28 155:195 156:254 157:254 158:254 159:254 160:254 161:255 162:61 181:6 182:191 183:253 184:253 185:253 186:253 187:253 188:253 189:253 190:60 208:26 209:190 210:253 211:253 212:253 213:253 214:240 215:191 216:242 217:253 218:60 235:15 236:187 237:253 238:253 239:253 240:253 241:253 242:200 244:211 245:253 246:60 262:22 263:66 264:253 265:253 266:253 267:253 268:241 269:209 270:44 271:23 272:218 273:253 274:60 290:124 291:253 292:253 293:253 294:253 295:253 296:182 299:131 300:253 301:253 302:60 318:38 319:217 320:253 321:253 322:244 323:111 324:37 327:131 328:253 329:253 330:60 346:124 347:253 348:253 349:253 350:165 354:22 355:182 356:253 357:253 358:60 374:124 375:253 376:253 377:240 378:45 382:53 383:253 384:253 385:249 386:58 401:16 402:168 403:253 404:216 405:45 410:53 411:253 412:253 413:138 429:159 430:253 431:253 432:147 438:53 439:253 440:253 441:138 456:136 457:252 458:253 459:227 460:5 466:53 467:253 468:243 469:101 484:140 485:253 486:253 487:124 494:156 495:253 496:218 511:13 512:164 513:253 514:142 515:5 521:32 522:233 523:253 524:218 539:62 540:253 541:253 542:130 548:37 549:203 550:253 551:253 552:127 567:62 568:253 569:253 570:147 571:36 572:36 573:36 574:36 575:151 576:222 577:253 578:245 579:127 580:8 595:34 596:202 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:200 624:140 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:248 633:235 634:65 652:87 653:173 654:253 655:253 656:253 657:253 658:253 659:253 660:182 681:14 682:78 683:96 684:253 685:253 686:253 687:137 688:56 +0 123:8 124:76 125:202 126:254 127:255 128:163 129:37 130:2 150:13 151:182 152:253 153:253 154:253 155:253 156:253 157:253 158:23 177:15 178:179 179:253 180:253 181:212 182:91 183:218 184:253 185:253 186:179 187:109 205:105 206:253 207:253 208:160 209:35 210:156 211:253 212:253 213:253 214:253 215:250 216:113 232:19 233:212 234:253 235:253 236:88 237:121 238:253 239:233 240:128 241:91 242:245 243:253 244:248 245:114 260:104 261:253 262:253 263:110 264:2 265:142 266:253 267:90 270:26 271:199 272:253 273:248 274:63 287:1 288:173 289:253 290:253 291:29 293:84 294:228 295:39 299:72 300:251 301:253 302:215 303:29 315:36 316:253 317:253 318:203 319:13 328:82 329:253 330:253 331:170 343:36 344:253 345:253 346:164 356:11 357:198 358:253 359:184 360:6 371:36 372:253 373:253 374:82 385:138 386:253 387:253 388:35 399:128 400:253 401:253 402:47 413:48 414:253 415:253 416:35 427:154 428:253 429:253 430:47 441:48 442:253 443:253 444:35 455:102 456:253 457:253 458:99 469:48 470:253 471:253 472:35 483:36 484:253 485:253 486:164 496:16 497:208 498:253 499:211 500:17 511:32 512:244 513:253 514:175 515:4 524:44 525:253 526:253 527:156 540:171 541:253 542:253 543:29 551:30 552:217 553:253 554:188 555:19 568:171 569:253 570:253 571:59 578:60 579:217 580:253 581:253 582:70 596:78 597:253 598:253 599:231 600:48 604:26 605:128 606:249 607:253 608:244 609:94 610:15 624:8 625:151 626:253 627:253 628:234 629:101 630:121 631:219 632:229 633:253 634:253 635:201 636:80 653:38 654:232 655:253 656:253 657:253 658:253 659:253 660:253 661:253 662:201 663:66 +0 127:68 128:254 129:255 130:254 131:107 153:11 154:176 155:230 156:253 157:253 158:253 159:212 180:28 181:197 182:253 183:253 184:253 185:253 186:253 187:229 188:107 189:14 208:194 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:253 217:53 235:69 236:241 237:253 238:253 239:253 240:253 241:241 242:186 243:253 244:253 245:195 262:10 263:161 264:253 265:253 266:253 267:246 268:40 269:57 270:231 271:253 272:253 273:195 290:140 291:253 292:253 293:253 294:253 295:154 297:25 298:253 299:253 300:253 301:195 318:213 319:253 320:253 321:253 322:135 323:8 325:3 326:128 327:253 328:253 329:195 345:77 346:238 347:253 348:253 349:253 350:7 354:116 355:253 356:253 357:195 372:11 373:165 374:253 375:253 376:231 377:70 378:1 382:78 383:237 384:253 385:195 400:33 401:253 402:253 403:253 404:182 411:200 412:253 413:195 428:98 429:253 430:253 431:253 432:24 439:42 440:253 441:195 456:197 457:253 458:253 459:253 460:24 467:163 468:253 469:195 484:197 485:253 486:253 487:189 488:13 494:53 495:227 496:253 497:121 512:197 513:253 514:253 515:114 521:21 522:227 523:253 524:231 525:27 540:197 541:253 542:253 543:114 547:5 548:131 549:143 550:253 551:231 552:59 568:197 569:253 570:253 571:236 572:73 573:58 574:217 575:223 576:253 577:253 578:253 579:174 596:197 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:48 624:149 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:182 634:15 635:3 652:12 653:168 654:253 655:253 656:253 657:253 658:253 659:248 660:89 661:23 +1 157:85 158:255 159:103 160:1 185:205 186:253 187:253 188:30 213:205 214:253 215:253 216:30 240:44 241:233 242:253 243:244 244:27 268:135 269:253 270:253 271:100 296:153 297:253 298:240 299:76 323:12 324:208 325:253 326:166 351:69 352:253 353:253 354:142 378:14 379:110 380:253 381:235 382:33 406:63 407:223 408:235 409:130 434:186 435:253 436:235 437:37 461:17 462:145 463:253 464:231 465:35 489:69 490:220 491:231 492:123 516:18 517:205 518:253 519:176 520:27 543:17 544:125 545:253 546:185 547:39 571:71 572:214 573:231 574:41 599:167 600:253 601:225 602:33 626:72 627:205 628:207 629:14 653:30 654:249 655:233 656:49 681:32 682:253 683:89 +1 126:94 127:132 154:250 155:250 156:4 182:250 183:254 184:95 210:250 211:254 212:95 238:250 239:254 240:95 266:250 267:254 268:95 294:250 295:254 296:95 322:250 323:254 324:95 350:250 351:254 352:95 378:250 379:254 380:95 405:77 406:254 407:250 408:19 433:96 434:254 435:249 461:53 462:253 463:252 464:43 490:250 491:251 492:32 517:85 518:254 519:249 545:96 546:254 547:249 573:83 574:254 575:250 576:14 602:250 603:254 604:95 630:250 631:255 632:95 658:132 659:254 660:95 +1 124:32 125:253 126:31 152:32 153:251 154:149 180:32 181:251 182:188 208:32 209:251 210:188 236:32 237:251 238:228 239:59 264:32 265:253 266:253 267:95 292:28 293:236 294:251 295:114 321:127 322:251 323:251 349:127 350:251 351:251 377:48 378:232 379:251 406:223 407:253 408:159 434:221 435:251 436:158 462:142 463:251 464:158 490:64 491:251 492:242 493:55 518:64 519:251 520:253 521:161 546:64 547:253 548:255 549:221 574:16 575:181 576:253 577:220 603:79 604:253 605:236 606:63 632:213 633:251 634:126 660:96 661:251 662:126 +1 129:39 130:254 131:255 132:254 133:140 157:136 158:253 159:253 160:228 161:67 184:6 185:227 186:253 187:253 188:58 211:29 212:188 213:253 214:253 215:253 216:17 239:95 240:253 241:253 242:253 243:157 244:8 266:3 267:107 268:253 269:253 270:245 271:77 294:29 295:253 296:253 297:240 298:100 322:141 323:253 324:253 325:215 349:129 350:248 351:253 352:253 353:215 377:151 378:253 379:253 380:253 381:144 405:151 406:253 407:253 408:253 409:27 431:3 432:102 433:242 434:253 435:253 436:110 437:3 459:97 460:253 461:253 462:253 463:214 464:55 487:207 488:253 489:253 490:253 491:158 515:67 516:253 517:253 518:253 519:158 543:207 544:253 545:253 546:240 547:88 571:207 572:253 573:253 574:224 598:32 599:217 600:253 601:253 602:224 626:141 627:253 628:253 629:253 630:133 654:36 655:219 656:253 657:140 658:10 +0 123:59 124:55 149:71 150:192 151:254 152:250 153:147 154:17 176:123 177:247 178:253 179:254 180:253 181:253 182:196 183:79 184:176 185:175 186:175 187:124 188:48 203:87 204:247 205:247 206:176 207:95 208:102 209:117 210:243 211:237 212:192 213:232 214:253 215:253 216:245 217:152 218:6 230:23 231:229 232:253 233:138 238:219 239:58 241:95 242:118 243:80 244:230 245:254 246:196 247:30 258:120 259:254 260:205 261:8 266:114 272:38 273:255 274:254 275:155 276:5 286:156 287:253 288:92 301:61 302:235 303:253 304:102 314:224 315:253 316:78 330:117 331:253 332:196 333:18 342:254 343:253 344:78 358:9 359:211 360:253 361:73 370:254 371:253 372:78 387:175 388:253 389:155 398:194 399:254 400:101 415:79 416:254 417:155 426:112 427:253 428:211 429:9 443:73 444:251 445:200 454:41 455:241 456:253 457:87 471:25 472:240 473:253 483:147 484:253 485:227 486:47 499:94 500:253 501:200 511:5 512:193 513:253 514:230 515:76 527:175 528:253 529:155 540:31 541:219 542:254 543:255 544:126 545:18 553:14 554:149 555:254 556:244 557:45 569:21 570:158 571:254 572:253 573:226 574:162 575:118 576:96 577:20 578:20 579:73 580:118 581:224 582:253 583:247 584:85 598:30 599:155 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:254 610:247 611:84 627:5 628:27 629:117 630:206 631:244 632:229 633:213 634:213 635:213 636:176 637:117 638:32 659:45 660:23 +1 128:58 129:139 156:247 157:247 158:25 183:121 184:253 185:156 186:3 211:133 212:253 213:145 238:11 239:227 240:253 241:145 266:7 267:189 268:253 269:145 294:35 295:252 296:253 297:145 322:146 323:252 324:253 325:131 350:146 351:252 352:253 353:13 378:146 379:252 380:253 381:13 406:147 407:253 408:255 409:13 434:146 435:252 436:253 437:13 462:146 463:252 464:253 465:13 490:146 491:252 492:253 493:13 517:22 518:230 519:252 520:221 521:9 545:22 546:230 547:252 548:133 574:146 575:252 576:133 602:146 603:252 604:120 630:146 631:252 658:146 659:252 +1 129:28 130:247 131:255 132:165 156:47 157:221 158:252 159:252 160:164 184:177 185:252 186:252 187:252 188:164 212:177 213:252 214:252 215:223 216:78 240:177 241:252 242:252 243:197 267:114 268:236 269:252 270:235 271:42 294:5 295:148 296:252 297:252 298:230 321:14 322:135 323:252 324:252 325:252 326:230 349:78 350:252 351:252 352:252 353:252 354:162 377:78 378:252 379:252 380:252 381:252 382:9 405:78 406:252 407:252 408:252 409:252 410:9 432:32 433:200 434:252 435:252 436:252 437:105 438:3 459:10 460:218 461:252 462:252 463:252 464:105 465:8 487:225 488:252 489:252 490:252 491:240 492:69 514:44 515:237 516:252 517:252 518:228 519:85 541:59 542:218 543:252 544:252 545:225 546:93 568:65 569:208 570:252 571:252 572:252 573:175 596:133 597:252 598:252 599:252 600:225 601:68 624:133 625:252 626:252 627:244 628:54 652:133 653:252 654:252 655:48 +0 156:13 157:6 181:10 182:77 183:145 184:253 185:190 186:67 207:11 208:77 209:193 210:252 211:252 212:253 213:252 214:238 215:157 216:71 217:26 233:10 234:78 235:193 236:252 237:252 238:252 239:252 240:253 241:252 242:252 243:252 244:252 245:228 246:128 247:49 248:5 259:6 260:78 261:194 262:252 263:252 264:252 265:252 266:252 267:252 268:253 269:217 270:192 271:232 272:252 273:252 274:252 275:252 276:135 277:3 286:4 287:147 288:252 289:252 290:252 291:252 292:252 293:252 294:252 295:252 296:175 297:26 299:40 300:145 301:235 302:252 303:252 304:252 305:104 314:208 315:252 316:252 317:252 318:252 319:252 320:252 321:133 322:48 323:48 329:71 330:236 331:252 332:252 333:230 342:253 343:185 344:170 345:252 346:252 347:252 348:173 349:22 358:102 359:252 360:252 361:252 370:24 371:141 372:243 373:252 374:252 375:186 376:5 386:8 387:220 388:252 389:252 398:70 399:247 400:252 401:252 402:165 403:37 414:81 415:251 416:252 417:194 426:255 427:253 428:253 429:251 430:69 441:39 442:231 443:253 444:253 445:127 454:253 455:252 456:249 457:127 468:6 469:147 470:252 471:252 472:190 473:5 482:253 483:252 484:216 495:7 496:145 497:252 498:252 499:252 500:69 510:253 511:252 512:223 513:16 522:25 523:185 524:252 525:252 526:252 527:107 528:8 538:167 539:252 540:252 541:181 542:18 549:105 550:191 551:252 552:252 553:235 554:151 555:10 566:37 567:221 568:252 569:252 570:210 571:193 572:96 573:73 574:130 575:188 576:194 577:227 578:252 579:252 580:235 581:128 595:97 596:220 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:253 605:252 606:252 607:236 608:70 624:40 625:174 626:252 627:252 628:252 629:252 630:252 631:252 632:253 633:197 634:138 635:29 653:5 654:23 655:116 656:143 657:143 658:143 659:143 660:24 661:10 +0 127:28 128:164 129:254 130:233 131:148 132:11 154:3 155:164 156:254 157:234 158:225 159:254 160:204 182:91 183:254 184:235 185:48 186:32 187:166 188:251 189:92 208:33 209:111 210:214 211:205 212:49 215:24 216:216 217:210 235:34 236:217 237:254 238:254 239:211 244:87 245:237 246:43 262:34 263:216 264:254 265:254 266:252 267:243 268:61 272:38 273:248 274:182 290:171 291:254 292:184 293:205 294:175 295:36 301:171 302:227 317:28 318:234 319:190 320:13 321:193 322:157 329:124 330:238 331:26 345:140 346:254 347:131 349:129 350:157 357:124 358:254 359:95 373:201 374:238 375:56 377:70 378:103 385:124 386:254 387:148 400:62 401:255 402:210 413:150 414:254 415:122 428:86 429:254 430:201 431:15 440:28 441:237 442:246 443:44 456:128 457:254 458:143 468:34 469:243 470:227 484:62 485:254 486:210 496:58 497:249 498:179 512:30 513:240 514:210 524:207 525:254 526:64 541:216 542:231 543:34 551:129 552:248 553:170 554:9 569:131 570:254 571:170 577:17 578:129 579:248 580:225 581:24 597:50 598:245 599:245 600:184 601:106 602:106 603:106 604:133 605:231 606:254 607:244 608:53 626:67 627:249 628:254 629:254 630:254 631:254 632:254 633:251 634:193 635:40 655:38 656:157 657:248 658:166 659:166 660:139 661:57 +0 129:105 130:255 131:219 132:67 133:67 134:52 156:20 157:181 158:253 159:253 160:253 161:253 162:226 163:69 182:4 183:129 184:206 185:253 186:253 187:253 188:253 189:253 190:253 191:130 209:9 210:141 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:166 220:20 237:134 238:253 239:253 240:253 241:253 242:253 243:253 244:253 245:253 246:253 247:253 248:65 262:2 263:83 264:207 265:246 266:253 267:253 268:253 269:253 270:253 271:249 272:234 273:247 274:253 275:253 276:65 290:83 291:253 292:253 293:253 294:253 295:253 296:189 297:253 298:253 299:205 301:179 302:253 303:253 304:65 317:85 318:234 319:253 320:253 321:253 322:253 323:157 324:26 325:164 326:151 327:83 329:179 330:253 331:253 332:65 344:65 345:237 346:253 347:253 348:253 349:67 350:36 351:14 353:15 354:12 357:179 358:253 359:253 360:65 371:4 372:141 373:253 374:253 375:221 376:158 377:23 385:179 386:253 387:253 388:65 399:129 400:253 401:253 402:241 403:62 412:72 413:226 414:253 415:175 416:24 426:119 427:247 428:253 429:253 430:206 439:8 440:134 441:253 442:253 443:130 454:132 455:253 456:253 457:194 458:27 467:125 468:253 469:253 470:253 471:130 481:45 482:213 483:253 484:253 485:112 493:70 494:170 495:247 496:253 497:253 498:89 499:43 509:67 510:253 511:253 512:196 513:55 514:9 520:8 521:131 522:253 523:253 524:253 525:86 526:1 537:67 538:253 539:253 540:253 541:253 542:129 546:43 547:114 548:134 549:253 550:253 551:231 552:139 553:41 565:20 566:167 567:253 568:253 569:253 570:247 571:179 572:179 573:179 574:206 575:253 576:253 577:253 578:253 579:72 594:103 595:240 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:244 605:119 606:8 607:1 623:107 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:175 632:111 651:3 652:121 653:253 654:253 655:253 656:253 657:253 658:182 659:24 +0 125:22 126:183 127:252 128:254 129:252 130:252 131:252 132:76 151:85 152:85 153:168 154:250 155:250 156:252 157:250 158:250 159:250 160:250 161:71 163:43 164:85 165:14 178:107 179:252 180:250 181:250 182:250 183:250 184:252 185:250 186:250 187:250 188:250 189:210 191:127 192:250 193:146 205:114 206:237 207:252 208:250 209:250 210:250 211:250 212:252 213:250 214:250 215:250 216:250 217:210 219:127 220:250 221:250 232:107 233:237 234:250 235:252 236:250 237:250 238:250 239:74 240:41 241:41 242:41 243:41 244:217 245:34 247:127 248:250 249:250 259:15 260:148 261:252 262:252 263:254 264:238 265:105 275:128 276:252 277:252 286:15 287:140 288:250 289:250 290:250 291:167 292:111 303:127 304:250 305:250 314:43 315:250 316:250 317:250 318:250 331:127 332:250 333:250 342:183 343:250 344:250 345:250 346:110 358:57 359:210 360:250 361:250 370:252 371:250 372:250 373:110 374:7 386:85 387:250 388:250 389:250 398:254 399:252 400:252 401:83 414:86 415:252 416:252 417:217 426:252 427:250 428:250 429:138 430:14 441:15 442:140 443:250 444:250 445:41 454:252 455:250 456:250 457:250 458:41 469:43 470:250 471:250 472:250 473:41 482:252 483:250 484:250 485:250 486:181 497:183 498:250 499:250 500:250 501:41 510:76 511:250 512:250 513:250 514:250 524:177 525:252 526:250 527:250 528:110 529:7 538:36 539:224 540:252 541:252 542:252 543:219 544:43 545:43 546:43 547:7 549:15 550:43 551:183 552:252 553:255 554:252 555:126 567:85 568:250 569:250 570:250 571:252 572:250 573:250 574:250 575:111 576:86 577:140 578:250 579:250 580:250 581:252 582:222 583:83 595:42 596:188 597:250 598:250 599:252 600:250 601:250 602:250 603:250 604:252 605:250 606:250 607:250 608:250 609:126 610:83 624:127 625:250 626:250 627:252 628:250 629:250 630:250 631:250 632:252 633:250 634:250 635:137 636:83 652:21 653:41 654:217 655:252 656:250 657:250 658:250 659:250 660:217 661:41 662:41 663:14 +1 155:114 156:206 157:25 183:238 184:252 185:55 211:222 212:252 213:55 239:113 240:252 241:55 267:113 268:252 269:55 295:255 296:253 297:56 323:253 324:176 325:6 350:32 351:253 352:233 353:43 378:140 379:253 380:195 381:19 406:140 407:253 408:167 433:29 434:253 435:141 461:29 462:252 463:140 489:29 490:252 491:140 517:29 518:252 519:140 545:29 546:252 547:140 573:169 574:253 575:79 601:169 602:252 628:76 629:234 630:141 656:197 657:233 658:37 684:197 685:223 +1 127:73 128:253 129:253 130:63 155:115 156:252 157:252 158:144 183:217 184:252 185:252 186:144 210:63 211:237 212:252 213:252 214:144 238:109 239:252 240:252 241:252 266:109 267:252 268:252 269:252 294:109 295:252 296:252 297:252 322:191 323:252 324:252 325:252 349:145 350:255 351:253 352:253 353:253 376:32 377:237 378:253 379:252 380:252 381:210 404:37 405:252 406:253 407:252 408:252 409:108 432:37 433:252 434:253 435:252 436:252 437:108 460:21 461:207 462:255 463:253 464:253 465:108 489:144 490:253 491:252 492:252 493:108 516:27 517:221 518:253 519:252 520:252 521:108 544:16 545:190 546:253 547:252 548:252 549:108 573:145 574:255 575:253 576:253 577:253 601:144 602:253 603:252 604:252 605:210 629:144 630:253 631:252 632:252 633:108 657:62 658:253 659:252 660:252 661:108 +1 120:85 121:253 122:132 123:9 147:82 148:241 149:251 150:251 151:128 175:175 176:251 177:251 178:251 179:245 180:121 203:13 204:204 205:251 206:251 207:251 208:245 209:107 232:39 233:251 234:251 235:251 236:251 237:167 238:22 260:15 261:155 262:251 263:251 264:251 265:251 266:177 289:15 290:157 291:248 292:251 293:251 294:251 295:165 319:214 320:251 321:251 322:251 323:212 324:78 325:24 347:109 348:251 349:251 350:251 351:253 352:251 353:170 354:10 375:5 376:57 377:162 378:251 379:253 380:251 381:251 382:18 405:106 406:239 407:255 408:253 409:253 410:213 434:105 435:253 436:251 437:251 438:230 439:72 463:253 464:251 465:251 466:251 467:221 468:67 491:72 492:251 493:251 494:251 495:251 496:96 519:36 520:199 521:251 522:251 523:251 524:155 525:15 548:45 549:204 550:251 551:251 552:251 553:157 577:161 578:249 579:251 580:251 581:248 582:147 606:233 607:251 608:251 609:251 610:173 634:233 635:251 636:251 637:251 638:173 662:53 663:131 664:251 665:251 666:173 +1 126:15 127:200 128:255 129:90 154:42 155:254 156:254 157:173 182:42 183:254 184:254 185:199 210:26 211:237 212:254 213:221 214:12 239:213 240:254 241:231 242:17 267:213 268:254 269:199 295:213 296:254 297:199 323:213 324:254 325:96 350:20 351:232 352:254 353:33 378:84 379:254 380:229 381:17 406:168 407:254 408:203 433:8 434:217 435:254 436:187 461:84 462:254 463:254 464:48 489:195 490:254 491:254 492:37 516:20 517:233 518:254 519:212 520:4 544:132 545:254 546:254 547:82 571:9 572:215 573:254 574:254 575:116 576:46 599:55 600:254 601:254 602:254 603:254 604:121 627:113 628:254 629:254 630:254 631:254 632:40 655:12 656:163 657:254 658:185 659:58 660:1 +0 182:32 183:57 184:57 185:57 186:57 187:57 188:57 189:57 208:67 209:185 210:229 211:252 212:252 213:252 214:253 215:252 216:252 217:252 218:185 219:66 234:13 235:188 236:246 237:252 238:253 239:252 240:252 241:252 242:241 243:139 244:177 245:252 246:253 247:246 248:187 249:13 261:26 262:255 263:253 264:244 265:175 266:101 274:126 275:244 276:253 277:153 288:82 289:243 290:253 291:214 292:81 303:169 304:252 305:252 315:19 316:215 317:252 318:206 319:56 331:169 332:252 333:252 343:157 344:252 345:252 346:13 359:169 360:252 361:151 370:41 371:253 372:253 373:128 386:92 387:253 388:206 389:13 398:166 399:252 400:196 401:9 414:216 415:252 416:142 426:253 427:252 428:168 441:89 442:253 443:208 444:13 454:253 455:252 456:68 468:38 469:225 470:253 471:96 482:254 483:253 484:56 495:45 496:229 497:253 498:151 510:253 511:252 512:81 522:70 523:225 524:252 525:227 538:216 539:252 540:168 548:29 549:134 550:253 551:252 552:186 553:31 566:91 567:252 568:243 569:125 573:51 574:114 575:113 576:210 577:252 578:253 579:151 580:19 595:157 596:253 597:253 598:254 599:253 600:253 601:253 602:254 603:253 604:244 605:175 606:51 623:19 624:122 625:196 626:197 627:221 628:196 629:196 630:197 631:121 632:56 655:25 +0 127:42 128:235 129:255 130:84 153:15 154:132 155:208 156:253 157:253 158:171 159:108 180:6 181:177 182:253 183:253 184:253 185:253 186:253 187:242 188:110 208:151 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:139 235:48 236:208 237:253 238:253 239:253 240:253 241:253 242:253 243:253 244:139 263:85 264:253 265:253 266:253 267:253 268:236 269:156 270:184 271:253 272:148 273:6 290:7 291:141 292:253 293:253 294:253 295:253 296:27 298:170 299:253 300:253 301:74 318:19 319:253 320:253 321:253 322:253 323:253 324:27 326:170 327:253 328:253 329:74 345:16 346:186 347:253 348:253 349:253 350:242 351:105 352:4 354:170 355:253 356:253 357:94 358:1 373:141 374:253 375:253 376:253 377:242 378:100 382:170 383:253 384:253 385:253 386:8 401:141 402:253 403:253 404:253 405:224 410:170 411:253 412:253 413:253 414:8 428:12 429:158 430:253 431:253 432:230 433:51 438:18 439:237 440:253 441:253 442:8 456:76 457:253 458:253 459:218 460:61 467:236 468:253 469:253 470:8 484:76 485:253 486:253 487:168 495:110 496:253 497:132 498:3 512:76 513:253 514:253 515:168 521:20 522:174 523:239 524:147 525:5 539:5 540:155 541:253 542:253 543:168 548:102 549:170 550:253 551:253 552:139 567:3 568:128 569:253 570:253 571:228 572:179 573:179 574:179 575:179 576:245 577:253 578:253 579:219 580:41 596:76 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:163 624:39 625:199 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:253 634:170 635:9 653:36 654:219 655:253 656:253 657:253 658:253 659:253 660:224 661:65 662:22 +1 156:202 157:253 158:69 184:253 185:252 186:121 212:253 213:252 214:69 240:253 241:252 242:69 267:106 268:253 269:231 270:37 295:179 296:255 297:196 322:17 323:234 324:253 325:92 350:93 351:252 352:253 353:92 378:93 379:252 380:253 381:92 406:93 407:252 408:232 409:8 434:208 435:253 436:116 462:207 463:252 464:116 490:207 491:252 492:32 517:57 518:244 519:252 545:122 546:252 547:252 573:185 574:253 575:253 601:184 602:252 603:252 629:101 630:252 631:252 657:13 658:173 659:252 660:43 686:9 687:232 688:116 +1 156:73 157:253 158:253 159:253 160:124 184:73 185:251 186:251 187:251 188:251 212:99 213:251 214:251 215:251 216:225 240:253 241:251 242:251 243:251 244:71 266:79 267:180 268:253 269:251 270:251 271:173 272:20 294:110 295:253 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:215 350:109 351:251 352:253 353:251 354:215 378:109 379:251 380:253 381:251 382:137 406:109 407:251 408:253 409:251 410:35 433:37 434:253 435:253 436:255 437:253 438:35 461:140 462:251 463:251 464:253 465:168 466:15 488:125 489:246 490:251 491:251 492:190 493:15 516:144 517:251 518:251 519:251 520:180 543:53 544:221 545:251 546:251 547:251 548:51 571:125 572:253 573:253 574:253 575:201 598:105 599:253 600:251 601:251 602:188 603:30 626:180 627:253 628:251 629:251 630:142 654:180 655:253 656:251 657:235 658:82 682:180 683:253 684:251 685:215 +1 124:111 125:255 126:48 152:162 153:253 154:237 155:63 180:206 181:253 182:253 183:183 208:87 209:217 210:253 211:205 237:90 238:253 239:238 240:60 265:37 266:225 267:253 268:89 294:206 295:253 296:159 322:206 323:253 324:226 350:206 351:253 352:226 378:206 379:253 380:226 406:206 407:253 408:226 434:206 435:253 436:226 462:206 463:253 464:226 490:206 491:253 492:226 518:206 519:253 520:237 521:45 546:206 547:253 548:253 549:109 574:173 575:253 576:253 577:109 602:69 603:253 604:253 605:109 630:64 631:248 632:253 633:109 659:112 660:253 661:109 +0 99:70 100:255 101:165 102:114 127:122 128:253 129:253 130:253 131:120 155:165 156:253 157:253 158:253 159:234 160:52 183:99 184:253 185:253 186:253 187:253 188:228 189:26 209:60 210:168 211:238 212:202 213:174 214:253 215:253 216:253 217:127 235:91 236:81 237:1 238:215 239:128 240:28 241:12 242:181 243:253 244:253 245:175 246:3 262:18 263:204 264:253 265:77 270:7 271:253 272:253 273:253 274:54 289:54 290:248 291:253 292:253 293:143 298:1 299:127 300:253 301:253 302:188 317:104 318:253 319:253 320:253 321:20 327:81 328:249 329:253 330:191 345:192 346:253 347:253 348:218 349:5 356:203 357:253 358:208 359:21 372:56 373:237 374:253 375:250 376:100 384:104 385:253 386:253 387:75 400:76 401:253 402:253 403:224 412:119 413:253 414:253 415:75 428:80 429:253 430:253 431:103 439:4 440:241 441:253 442:218 443:32 456:213 457:253 458:253 459:103 467:125 468:253 469:253 470:191 484:213 485:253 486:253 487:103 494:3 495:176 496:253 497:253 498:135 512:213 513:253 514:253 515:103 521:9 522:162 523:253 524:253 525:226 526:37 540:179 541:253 542:253 543:135 548:46 549:157 550:253 551:253 552:253 553:63 568:23 569:188 570:253 571:249 572:179 573:179 574:179 575:179 576:233 577:253 578:253 579:233 580:156 581:10 597:51 598:235 599:253 600:253 601:253 602:253 603:253 604:253 605:251 606:232 607:120 626:16 627:124 628:253 629:253 630:253 631:253 632:152 633:104 +1 124:29 125:197 126:255 127:84 152:85 153:251 154:253 155:83 180:86 181:253 182:254 183:253 208:85 209:251 210:253 211:251 236:86 237:253 238:254 239:253 240:169 264:85 265:251 266:253 267:251 268:168 292:86 293:253 294:254 295:253 296:169 320:28 321:196 322:253 323:251 324:168 349:169 350:254 351:253 352:169 377:168 378:253 379:251 380:168 405:169 406:254 407:253 408:169 433:168 434:253 435:251 436:168 462:254 463:253 464:254 465:139 490:253 491:251 492:253 493:251 518:254 519:253 520:254 521:253 522:57 546:253 547:251 548:253 549:251 550:168 574:198 575:253 576:254 577:253 578:114 602:85 603:251 604:253 605:251 630:85 631:253 632:254 633:253 658:28 659:83 660:196 661:83 +1 159:31 160:210 161:253 162:163 187:198 188:252 189:252 190:162 213:10 214:86 215:242 216:252 217:252 218:66 241:164 242:252 243:252 244:252 245:188 246:8 268:53 269:242 270:252 271:252 272:225 273:14 296:78 297:252 298:252 299:252 300:204 323:56 324:231 325:252 326:252 327:212 328:35 351:157 352:252 353:252 354:252 355:37 377:8 378:132 379:253 380:252 381:252 382:230 383:24 405:45 406:252 407:253 408:252 409:154 410:55 427:7 428:55 433:107 434:253 435:255 436:228 437:53 454:15 455:24 456:23 460:110 461:242 462:252 463:228 464:59 482:57 483:83 487:88 488:247 489:252 490:252 491:140 514:15 515:189 516:252 517:252 518:252 542:74 543:252 544:252 545:238 546:90 570:178 571:252 572:252 573:189 597:40 598:217 599:252 600:252 601:59 625:75 626:252 627:252 628:252 629:85 630:61 653:62 654:239 655:252 656:156 657:14 682:178 683:252 684:14 +1 131:159 132:255 133:122 158:167 159:228 160:253 161:121 185:64 186:236 187:251 188:205 189:110 212:48 213:158 214:251 215:251 216:178 217:39 240:190 241:251 242:251 243:251 267:96 268:253 269:253 270:253 271:153 295:194 296:251 297:251 298:211 299:74 322:80 323:174 324:251 325:251 326:140 327:47 349:16 350:181 351:253 352:251 353:219 354:23 377:64 378:251 379:253 380:251 381:204 382:19 405:223 406:253 407:255 408:233 409:48 431:20 432:174 433:244 434:251 435:253 436:109 437:31 459:96 460:189 461:251 462:251 463:126 464:31 486:24 487:106 488:251 489:235 490:188 491:100 514:96 515:251 516:251 517:228 518:59 542:255 543:253 544:253 545:213 546:36 569:100 570:253 571:251 572:251 573:85 574:23 596:32 597:127 598:253 599:235 600:126 601:15 624:104 625:251 626:253 627:240 628:79 652:83 653:193 654:253 655:220 +0 153:92 154:191 155:178 156:253 157:242 158:141 159:104 160:29 180:26 181:253 182:252 183:252 184:252 185:253 186:252 187:252 188:252 189:108 190:19 206:57 207:123 208:222 209:253 210:252 211:252 212:252 213:168 214:224 215:252 216:252 217:253 218:84 233:176 234:243 235:252 236:252 237:253 238:252 239:252 240:252 242:19 243:153 244:252 245:253 246:209 247:25 259:10 260:128 261:255 262:253 263:244 264:225 265:114 266:194 267:253 268:178 272:163 273:254 274:253 275:168 287:85 288:252 289:253 290:189 291:56 294:19 295:133 296:9 300:38 301:253 302:252 303:168 314:19 315:191 316:252 317:194 318:19 329:253 330:252 331:234 332:22 342:107 343:252 344:252 345:13 357:253 358:252 359:252 360:128 370:169 371:253 372:241 385:141 386:253 387:253 388:140 397:19 398:225 399:252 400:139 413:66 414:252 415:252 416:139 425:29 426:252 427:252 428:52 441:29 442:252 443:252 444:139 453:29 454:252 455:252 456:28 469:29 470:252 471:252 472:40 481:141 482:253 483:253 484:91 497:154 498:253 499:168 509:66 510:252 511:252 512:165 525:253 526:252 527:168 537:19 538:224 539:252 540:252 552:126 553:253 554:252 555:80 566:169 567:252 568:252 569:214 570:38 579:126 580:249 581:253 582:151 583:6 594:26 595:223 596:253 597:254 598:253 599:128 600:29 604:13 605:41 606:216 607:253 608:253 609:226 610:38 623:122 624:252 625:253 626:252 627:252 628:252 629:169 630:169 631:169 632:206 633:253 634:252 635:252 636:202 637:38 651:19 652:56 653:168 654:224 655:252 656:252 657:253 658:252 659:252 660:252 661:253 662:233 663:130 664:6 682:94 683:139 684:190 685:153 686:252 687:164 688:139 689:28 690:22 +1 128:53 129:250 130:255 131:25 156:167 157:253 158:253 159:25 182:3 183:123 184:247 185:253 186:253 187:25 210:9 211:253 212:253 213:253 214:253 215:25 238:9 239:253 240:253 241:253 242:253 243:25 266:9 267:253 268:253 269:253 270:180 271:13 294:9 295:253 296:253 297:253 298:104 322:9 323:253 324:253 325:253 326:104 350:15 351:253 352:253 353:253 354:104 378:184 379:253 380:253 381:228 382:68 406:184 407:253 408:253 409:182 433:103 434:251 435:253 436:253 437:12 461:106 462:253 463:253 464:253 465:8 488:24 489:238 490:253 491:253 492:253 493:8 516:27 517:253 518:253 519:253 520:253 521:8 544:27 545:253 546:253 547:253 548:253 549:8 572:27 573:253 574:253 575:253 576:177 577:4 600:160 601:253 602:253 603:253 604:87 628:202 629:253 630:253 631:219 632:54 656:81 657:253 658:247 659:51 +0 122:63 123:176 124:253 125:253 126:159 127:113 128:63 150:140 151:253 152:252 153:252 154:252 155:252 156:241 157:100 158:66 177:54 178:227 179:253 180:252 181:252 182:252 183:252 184:253 185:252 186:239 187:181 188:57 204:38 205:224 206:252 207:253 208:226 209:246 210:252 211:252 212:253 213:252 214:252 215:252 216:252 217:108 218:3 232:57 233:252 234:252 235:253 236:27 237:88 238:112 239:112 240:112 241:112 242:142 243:252 244:252 245:253 246:152 247:31 260:198 261:253 262:253 263:79 270:32 271:153 272:253 273:255 274:253 275:196 287:76 288:246 289:252 290:127 299:3 300:106 301:253 302:252 303:214 304:28 315:194 316:252 317:252 318:112 329:143 330:252 331:252 332:193 343:225 344:252 345:217 346:37 357:38 358:234 359:252 360:223 370:63 371:240 372:252 373:84 386:146 387:252 388:223 398:114 399:253 400:228 401:47 414:147 415:253 416:253 417:112 426:159 427:252 428:195 442:225 443:252 444:252 445:112 454:253 455:252 456:195 470:225 471:252 472:230 473:25 482:159 483:252 484:202 485:10 497:92 498:243 499:252 500:208 510:113 511:252 512:252 513:161 524:79 525:253 526:252 527:220 528:37 538:114 539:253 540:253 541:253 542:174 543:63 550:26 551:128 552:253 553:255 554:253 555:133 566:12 567:228 568:252 569:252 570:252 571:241 572:100 573:85 574:76 576:85 577:131 578:231 579:252 580:252 581:253 582:129 583:6 595:97 596:208 597:252 598:252 599:253 600:252 601:252 602:246 603:197 604:253 605:252 606:252 607:252 608:220 609:133 610:6 624:19 625:99 626:239 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:245 635:223 636:99 654:63 655:112 656:112 657:221 658:252 659:252 660:253 661:127 662:87 +0 153:12 154:136 155:254 156:255 157:195 158:115 159:3 180:6 181:175 182:253 183:196 184:160 185:252 186:253 187:15 208:130 209:253 210:234 211:4 213:27 214:205 215:232 216:40 235:54 236:246 237:253 238:68 242:24 243:243 244:106 262:3 263:134 264:235 265:99 266:4 271:132 272:247 273:77 290:56 291:253 292:62 299:23 300:233 301:129 318:179 319:183 320:4 328:182 329:220 345:21 346:232 347:59 356:95 357:232 358:21 373:128 374:183 385:228 386:85 401:187 402:124 413:228 414:186 429:187 430:124 441:228 442:104 457:187 458:124 469:169 470:184 485:187 486:124 497:203 498:150 513:187 514:124 524:10 525:220 526:39 541:187 542:155 552:111 553:201 569:129 570:228 571:7 579:12 580:181 581:76 598:234 599:166 600:9 606:24 607:209 608:106 626:139 627:250 628:167 629:11 630:2 631:11 632:11 633:129 634:227 635:90 636:11 655:95 656:247 657:253 658:178 659:253 660:253 661:244 662:86 684:47 685:175 686:253 687:232 688:149 689:40 +1 128:255 129:253 130:57 156:253 157:251 158:225 159:56 183:169 184:254 185:253 186:254 187:84 211:168 212:253 213:251 214:253 215:83 238:85 239:253 240:254 241:253 242:169 266:85 267:251 268:253 269:251 270:56 294:141 295:253 296:254 297:253 322:253 323:251 324:253 325:251 350:254 351:253 352:254 353:253 378:253 379:251 380:253 381:251 406:254 407:253 408:254 409:196 433:114 434:253 435:251 436:253 437:83 461:169 462:254 463:253 464:226 465:56 489:168 490:253 491:251 492:168 516:85 517:253 518:254 519:253 544:85 545:251 546:253 547:251 572:254 573:253 574:254 575:253 600:253 601:251 602:253 603:251 628:254 629:253 630:254 631:253 656:139 657:251 658:253 659:138 +0 151:23 152:167 153:208 154:254 155:255 156:129 157:19 179:151 180:253 181:253 182:253 183:253 184:253 185:209 186:26 207:181 208:253 209:253 210:253 211:227 212:181 213:253 214:207 215:22 235:227 236:253 237:253 238:253 239:92 240:38 241:226 242:253 243:129 244:2 263:193 264:253 265:253 266:248 267:62 269:50 270:253 271:253 272:45 291:170 292:253 293:253 294:135 297:12 298:208 299:253 300:119 318:16 319:232 320:253 321:253 322:21 326:60 327:253 328:185 346:164 347:253 348:253 349:224 350:14 354:14 355:217 356:247 357:62 373:3 374:193 375:253 376:250 377:64 383:199 384:253 385:179 401:67 402:253 403:253 404:205 411:98 412:253 413:188 429:151 430:253 431:245 432:43 439:63 440:250 441:188 457:151 458:253 459:243 468:244 469:222 470:22 485:151 486:253 487:217 496:244 497:253 498:115 512:3 513:195 514:253 515:134 524:156 525:253 526:150 541:140 542:253 543:134 552:239 553:253 554:139 569:44 570:253 571:134 579:53 580:246 581:237 582:32 597:8 598:200 599:229 600:40 606:25 607:225 608:253 609:188 626:120 627:250 628:230 629:58 630:17 632:12 633:42 634:213 635:253 636:238 637:84 655:151 656:253 657:253 658:217 659:179 660:206 661:253 662:253 663:196 664:118 683:18 684:58 685:145 686:152 687:253 688:214 689:145 690:74 691:7 +1 130:24 131:150 132:233 133:38 156:14 157:89 158:253 159:254 160:254 161:71 183:78 184:203 185:254 186:254 187:254 188:232 189:77 190:54 191:8 209:12 210:155 211:240 212:254 213:223 214:76 215:254 216:254 217:254 218:254 219:68 235:3 236:101 237:216 238:254 239:227 240:122 241:26 242:110 243:254 244:254 245:254 246:184 247:100 262:46 263:222 264:254 265:254 266:179 267:48 270:181 271:254 272:254 273:146 274:6 288:2 289:145 290:248 291:254 292:182 293:111 294:4 297:3 298:250 299:254 300:206 301:3 315:6 316:144 317:254 318:254 319:171 325:125 326:254 327:252 328:80 342:6 343:142 344:254 345:179 346:95 347:4 352:61 353:246 354:254 355:150 370:64 371:254 372:177 373:14 380:124 381:254 382:246 383:32 398:108 399:97 400:15 407:24 408:226 409:254 410:116 435:177 436:255 437:254 438:5 463:196 464:254 465:99 466:1 490:3 491:199 492:254 493:79 518:129 519:254 520:254 521:23 546:178 547:254 548:192 549:8 550:3 551:43 573:11 574:198 575:254 576:128 577:66 578:130 579:225 595:137 596:202 597:106 598:84 599:84 600:84 601:112 602:254 603:254 604:254 605:254 606:212 607:151 623:172 624:254 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:254 633:162 634:75 651:12 652:106 653:177 654:254 655:254 656:254 657:235 658:135 659:100 660:17 661:2 +0 125:120 126:253 127:253 128:63 151:38 152:131 153:246 154:252 155:252 156:203 157:15 179:222 180:252 181:252 182:252 183:252 184:166 185:38 205:4 206:107 207:253 208:252 209:252 210:252 211:252 212:253 213:224 214:137 215:26 233:107 234:252 235:253 236:252 237:220 238:128 239:252 240:253 241:252 242:252 243:239 244:140 261:170 262:253 263:255 264:168 267:79 268:192 269:253 270:253 271:253 272:253 273:255 274:90 288:51 289:243 290:252 291:215 292:33 296:12 297:74 298:233 299:252 300:252 301:253 302:195 303:19 316:166 317:252 318:252 319:31 326:43 327:149 328:195 329:253 330:252 331:177 332:19 343:57 344:234 345:252 346:252 357:237 358:252 359:252 360:180 361:13 371:85 372:252 373:252 374:173 385:50 386:237 387:252 388:252 389:112 399:226 400:253 401:240 402:63 414:163 415:253 416:253 417:112 426:38 427:234 428:252 429:176 442:85 443:252 444:252 445:158 454:113 455:252 456:252 457:84 470:19 471:209 472:252 473:252 482:207 483:252 484:252 485:84 498:10 499:203 500:252 501:236 510:253 511:252 512:252 513:84 526:85 527:252 528:252 529:112 538:114 539:253 540:253 541:146 553:51 554:159 555:253 556:240 557:63 566:75 567:243 568:252 569:249 570:146 579:57 580:85 581:238 582:252 583:252 584:99 595:116 596:252 597:252 598:252 599:198 600:197 601:165 602:57 603:57 604:57 605:182 606:197 607:234 608:252 609:253 610:233 611:164 612:19 623:28 624:84 625:180 626:252 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:252 635:252 636:252 637:225 638:71 653:13 654:112 655:253 656:252 657:252 658:252 659:252 660:253 661:252 662:252 663:157 664:112 +1 127:155 128:253 129:126 155:253 156:251 157:141 158:4 183:253 184:251 185:251 186:31 211:253 212:251 213:251 214:31 239:253 240:251 241:251 242:31 267:255 268:253 269:253 270:31 293:8 294:131 295:253 296:251 297:235 298:27 321:64 322:251 323:253 324:251 325:126 349:64 350:251 351:253 352:251 353:126 377:64 378:251 379:253 380:251 381:126 405:64 406:253 407:255 408:221 433:182 434:251 435:253 436:200 460:64 461:236 462:251 463:253 464:62 487:8 488:158 489:251 490:251 491:169 492:8 515:32 516:251 517:251 518:251 519:158 543:32 544:253 545:253 546:253 547:159 571:32 572:251 573:251 574:251 575:39 599:32 600:251 601:251 602:251 627:32 628:251 629:251 630:251 631:100 655:32 656:251 657:251 658:251 +0 101:88 102:127 103:5 126:19 127:58 128:20 129:14 130:217 131:19 152:7 153:146 154:247 155:253 156:235 157:27 158:84 159:81 180:126 181:253 182:164 183:19 184:15 187:156 188:9 208:214 209:222 210:34 215:234 216:58 235:59 236:254 237:116 243:235 244:58 263:141 264:251 265:72 271:151 272:140 291:224 292:233 299:136 300:223 319:254 320:218 327:136 328:253 347:254 348:135 355:136 356:253 374:23 375:255 376:114 383:137 384:231 402:98 403:254 404:122 411:136 412:155 430:98 431:254 432:106 439:166 440:155 458:98 459:254 460:128 467:234 468:193 486:98 487:254 488:135 494:61 495:248 496:118 515:255 516:238 517:18 521:13 522:224 523:254 524:58 543:201 544:253 545:128 546:2 548:5 549:150 550:253 551:167 552:9 571:18 572:226 573:253 574:49 575:31 576:156 577:253 578:228 579:13 600:147 601:253 602:243 603:241 604:254 605:227 606:43 628:5 629:126 630:245 631:253 632:231 633:46 +0 127:37 128:141 129:156 130:156 131:194 132:194 133:47 153:11 154:132 155:239 156:253 157:253 158:253 159:253 160:254 161:181 180:25 181:172 182:253 183:235 184:167 185:78 186:93 187:174 188:254 189:247 190:54 207:26 208:210 209:253 210:237 211:90 216:201 217:253 218:78 235:192 236:253 237:237 238:58 244:156 245:253 246:78 262:141 263:254 264:235 265:53 269:19 270:5 272:156 273:254 274:78 289:46 290:254 291:253 292:92 296:17 297:226 298:217 299:49 300:148 301:253 302:78 317:165 318:254 319:239 320:24 324:20 325:253 326:253 327:58 328:18 329:115 330:24 344:37 345:248 346:254 347:91 352:2 353:117 354:250 355:163 356:91 372:77 373:253 374:254 375:39 382:196 383:253 384:173 400:159 401:254 402:218 403:15 410:77 411:254 412:255 413:61 428:234 429:253 430:113 438:21 439:226 440:254 441:135 455:25 456:240 457:253 458:68 467:195 468:254 469:135 483:79 484:253 485:253 495:195 496:254 497:135 511:79 512:253 513:253 514:76 523:195 524:254 525:99 540:212 541:254 542:209 543:9 550:10 551:209 552:196 553:15 568:54 569:253 570:254 571:137 572:36 576:2 577:20 578:168 579:253 580:60 596:28 597:235 598:254 599:253 600:199 601:124 602:79 603:79 604:167 605:253 606:253 607:185 608:30 625:15 626:117 627:217 628:253 629:253 630:253 631:254 632:253 633:240 634:109 635:12 655:27 656:126 657:208 658:253 659:193 660:147 661:40 +0 154:32 155:134 156:218 157:254 158:254 159:254 160:217 161:84 176:44 177:208 178:215 179:156 180:35 181:119 182:236 183:246 184:136 185:91 186:69 187:151 188:249 189:246 190:78 203:44 204:230 205:254 206:254 207:254 208:254 209:254 210:196 211:48 216:60 217:224 218:210 219:24 231:118 232:254 233:202 234:19 235:201 236:254 237:181 238:9 245:35 246:233 247:168 259:193 260:223 261:34 263:59 264:163 265:236 266:15 274:140 275:205 276:8 286:60 287:254 288:176 293:38 302:54 303:237 304:80 314:59 315:254 316:93 331:131 332:200 342:59 343:240 344:24 359:79 360:214 370:59 371:234 387:67 388:248 389:54 398:59 399:234 416:235 417:58 426:60 427:235 443:79 444:255 445:59 454:59 455:251 456:66 471:79 472:250 473:54 482:59 483:254 484:108 499:146 500:214 510:5 511:203 512:187 513:3 526:4 527:188 528:199 539:118 540:254 541:57 554:96 555:254 556:117 567:16 568:237 569:224 570:14 581:14 582:187 583:206 584:8 596:88 597:252 598:186 599:16 608:16 609:187 610:252 611:125 625:100 626:254 627:237 628:94 629:24 635:13 636:214 637:254 638:166 653:3 654:57 655:215 656:248 657:241 658:235 659:197 660:137 661:137 662:137 663:231 664:238 665:155 666:25 684:57 685:155 686:246 687:254 688:254 689:254 690:254 691:147 692:36 +1 124:102 125:252 126:252 127:41 152:102 153:250 154:250 155:202 180:102 181:250 182:250 183:232 184:91 208:102 209:250 210:250 211:212 212:29 236:102 237:252 238:252 239:254 240:150 264:102 265:250 266:250 267:252 268:149 292:102 293:250 294:250 295:252 296:149 320:102 321:250 322:250 323:252 324:231 325:80 349:152 350:252 351:254 352:252 353:100 377:151 378:250 379:252 380:250 381:100 405:151 406:250 407:252 408:250 409:100 433:151 434:250 435:252 436:250 437:100 461:123 462:243 463:254 464:252 465:100 490:202 491:252 492:250 493:100 518:80 519:252 520:250 521:190 522:30 547:252 548:250 549:250 550:49 575:255 576:252 577:252 578:252 579:214 580:31 603:171 604:250 605:250 606:250 607:252 608:190 609:40 631:20 632:160 633:250 634:250 635:252 636:250 637:100 660:20 661:170 662:250 663:212 664:49 665:20 +0 124:20 125:121 126:197 127:253 128:64 151:23 152:200 153:252 154:252 155:252 156:184 157:6 178:25 179:197 180:252 181:252 182:252 183:252 184:253 185:228 186:107 187:15 205:26 206:196 207:252 208:252 209:252 210:252 211:252 212:253 213:252 214:252 215:219 216:178 217:21 233:186 234:252 235:238 236:94 237:67 238:224 239:217 240:53 241:109 242:245 243:252 244:252 245:213 246:63 260:98 261:242 262:252 263:101 266:39 267:31 270:109 271:128 272:241 273:252 274:207 275:97 287:17 288:230 289:252 290:241 291:56 300:109 301:252 302:252 303:229 304:17 314:13 315:192 316:252 317:243 318:96 328:25 329:127 330:252 331:252 332:120 342:121 343:252 344:252 345:165 357:125 358:252 359:252 360:246 361:70 370:190 371:252 372:252 373:39 385:26 386:210 387:252 388:252 389:119 398:255 399:253 400:159 414:22 415:209 416:253 417:183 426:253 427:252 428:103 443:34 444:252 445:252 454:253 455:252 456:26 471:27 472:252 473:252 482:253 483:252 484:168 485:13 499:70 500:252 501:209 510:147 511:252 512:252 513:75 526:68 527:233 528:252 529:119 538:121 539:252 540:252 541:189 542:40 552:15 553:82 554:231 555:252 556:214 557:31 566:38 567:135 568:248 569:252 570:231 571:145 572:41 573:41 574:41 575:41 576:20 577:24 578:37 579:83 580:194 581:252 582:252 583:212 584:33 596:83 597:213 598:252 599:252 600:252 601:252 602:252 603:252 604:204 605:213 606:243 607:252 608:252 609:252 610:212 611:34 625:34 626:140 627:238 628:248 629:252 630:252 631:252 632:253 633:252 634:252 635:241 636:238 637:238 638:75 656:82 657:119 658:119 659:119 660:120 661:119 662:119 663:19 +1 127:20 128:254 129:255 130:37 155:19 156:253 157:253 158:134 183:19 184:253 185:253 186:246 187:125 211:76 212:253 213:253 214:253 215:158 239:207 240:253 241:253 242:253 243:158 267:207 268:253 269:253 270:253 271:158 294:48 295:223 296:253 297:253 298:243 299:106 322:141 323:253 324:253 325:253 326:113 349:65 350:237 351:253 352:253 353:253 354:36 377:76 378:253 379:253 380:253 381:253 382:36 405:76 406:253 407:253 408:253 409:253 410:36 433:76 434:253 435:253 436:253 437:118 438:4 460:4 461:148 462:253 463:253 464:253 465:103 488:10 489:253 490:253 491:253 492:253 493:103 516:10 517:253 518:253 519:253 520:173 521:7 544:10 545:253 546:253 547:253 548:168 572:143 573:253 574:253 575:239 576:49 600:198 601:253 602:253 603:234 615:140 628:198 629:253 630:253 631:234 656:198 657:253 658:253 659:234 +0 235:40 236:37 238:7 239:77 240:137 241:136 242:136 243:136 244:136 245:40 246:6 261:16 262:135 263:254 264:233 266:152 267:215 268:96 269:140 270:155 271:118 272:230 273:254 274:158 275:68 288:19 289:164 290:254 291:114 294:235 295:140 301:99 302:230 303:254 304:186 305:14 315:70 316:226 317:242 318:121 322:104 323:195 324:38 330:33 331:179 332:253 333:140 342:41 343:241 344:198 345:43 359:24 360:209 361:223 370:164 371:250 372:66 388:136 389:253 398:254 399:158 416:136 417:215 426:255 427:76 442:5 443:127 444:246 445:133 454:254 455:122 469:5 470:150 471:247 472:91 473:9 482:254 483:165 495:13 496:79 497:194 498:216 499:84 510:111 511:251 512:87 519:16 520:25 521:40 522:107 523:186 524:213 525:117 526:25 538:14 539:185 540:235 541:142 542:23 546:91 547:157 548:231 549:207 550:126 551:49 569:143 570:195 571:255 572:254 573:254 574:244 575:157 576:76 599:39 600:39 601:39 602:33 +1 128:166 129:255 130:187 131:6 156:165 157:253 158:253 159:13 183:15 184:191 185:253 186:253 187:13 211:49 212:253 213:253 214:253 215:13 239:141 240:253 241:253 242:169 243:4 266:4 267:189 268:253 269:249 270:53 294:69 295:253 296:253 297:246 322:69 323:253 324:253 325:246 350:118 351:253 352:253 353:124 378:206 379:253 380:231 381:21 405:66 406:241 407:253 408:199 433:105 434:253 435:253 436:89 460:3 461:228 462:253 463:252 464:86 488:111 489:253 490:253 491:205 516:166 517:253 518:253 519:75 543:43 544:249 545:253 546:193 547:9 570:4 571:160 572:253 573:253 574:184 598:37 599:253 600:253 601:253 602:88 626:140 627:253 628:253 629:186 630:18 654:14 655:253 656:253 657:27 +1 128:117 129:128 155:2 156:199 157:127 183:81 184:254 185:87 211:116 212:254 213:48 239:175 240:234 241:18 266:5 267:230 268:168 294:80 295:255 296:142 322:80 323:255 324:142 350:80 351:251 352:57 378:129 379:239 406:164 407:209 433:28 434:245 435:159 461:64 462:254 463:144 489:84 490:254 491:80 517:143 518:254 519:30 544:3 545:225 546:200 572:48 573:254 574:174 600:48 601:254 602:174 628:93 629:254 630:129 656:53 657:234 658:41 +1 129:159 130:142 156:11 157:220 158:141 184:78 185:254 186:141 212:111 213:254 214:109 240:196 241:221 242:15 267:26 268:221 269:159 295:63 296:254 297:159 323:178 324:254 325:93 350:7 351:191 352:254 353:97 378:42 379:255 380:254 381:41 406:42 407:254 408:195 409:10 434:141 435:255 436:78 461:11 462:202 463:254 464:59 489:86 490:254 491:254 492:59 517:142 518:254 519:248 520:52 545:142 546:254 547:195 573:142 574:254 575:164 601:142 602:254 603:77 629:142 630:254 631:131 657:77 658:172 659:5 +0 124:66 125:254 126:254 127:58 128:60 129:59 130:59 131:50 151:73 152:233 153:253 154:253 155:148 156:254 157:253 158:253 159:232 160:73 179:156 180:253 181:253 182:253 183:117 184:255 185:253 186:253 187:253 188:223 189:176 190:162 205:37 206:116 207:246 208:253 209:180 210:18 211:4 212:18 213:109 214:241 215:253 216:253 217:253 218:236 219:28 233:235 234:253 235:253 236:245 237:107 242:109 243:170 244:253 245:253 246:253 247:174 261:235 262:253 263:253 264:233 271:15 272:156 273:253 274:253 275:223 276:72 287:10 288:156 289:250 290:253 291:253 292:67 300:99 301:253 302:253 303:253 304:127 305:5 315:118 316:253 317:253 318:253 319:204 320:26 328:68 329:223 330:253 331:253 332:253 333:57 342:32 343:191 344:253 345:253 346:253 347:97 357:156 358:253 359:253 360:253 361:57 370:59 371:253 372:253 373:253 374:253 375:97 385:36 386:224 387:253 388:253 389:57 398:60 399:254 400:255 401:254 402:156 413:37 414:226 415:254 416:254 417:58 426:59 427:253 428:253 429:253 430:154 441:156 442:253 443:253 444:253 445:57 454:59 455:253 456:253 457:253 458:154 469:156 470:253 471:253 472:253 473:57 482:59 483:253 484:253 485:253 486:246 487:90 496:16 497:171 498:253 499:253 500:231 501:49 510:59 511:253 512:253 513:253 514:253 515:156 516:91 524:99 525:253 526:253 527:222 528:71 538:59 539:253 540:253 541:253 542:253 543:253 544:245 545:109 551:145 552:194 553:253 554:253 555:174 566:9 567:38 568:174 569:251 570:253 571:253 572:253 573:241 574:215 575:215 576:217 577:215 578:215 579:250 580:253 581:253 582:221 583:26 597:235 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:204 610:26 625:108 626:116 627:200 628:253 629:253 630:253 631:253 632:254 633:253 634:253 635:253 636:199 637:44 655:36 656:57 657:118 658:253 659:253 660:58 661:57 662:57 663:57 664:35 +1 129:101 130:222 131:84 157:225 158:252 159:84 184:89 185:246 186:208 187:19 212:128 213:252 214:195 239:79 240:253 241:252 242:195 267:141 268:255 269:253 270:133 294:26 295:240 296:253 297:252 298:55 322:60 323:252 324:253 325:154 326:12 349:7 350:178 351:252 352:253 353:27 377:57 378:252 379:252 380:253 381:27 405:57 406:253 407:253 408:204 409:15 433:104 434:252 435:252 436:94 460:19 461:209 462:252 463:252 488:101 489:252 490:252 491:157 516:225 517:252 518:252 519:112 544:226 545:253 546:240 547:63 572:225 573:252 574:223 600:225 601:252 602:223 628:225 629:252 630:242 631:75 656:146 657:252 658:236 659:50 +0 124:41 125:254 126:254 127:157 128:34 129:34 130:218 131:255 132:206 133:34 134:18 151:53 152:238 153:252 154:252 155:252 156:252 157:252 158:252 159:252 160:252 161:252 162:162 163:26 178:66 179:220 180:252 181:252 182:252 183:209 184:153 185:223 186:252 187:252 188:252 189:252 190:252 191:98 206:166 207:252 208:252 209:252 210:252 211:141 213:85 214:230 215:252 216:252 217:252 218:252 219:98 234:166 235:252 236:252 237:252 238:252 239:141 242:73 243:102 244:252 245:252 246:252 247:98 262:166 263:252 264:252 265:252 266:191 267:30 271:5 272:97 273:252 274:252 275:220 276:51 289:123 290:245 291:252 292:252 293:202 294:14 300:56 301:252 302:252 303:252 304:65 316:18 317:154 318:252 319:252 320:241 328:56 329:252 330:252 331:252 332:65 343:21 344:146 345:252 346:252 347:252 348:241 356:56 357:252 358:252 359:252 360:65 371:67 372:252 373:252 374:252 375:252 376:241 384:56 385:252 386:252 387:252 388:65 399:67 400:252 401:252 402:252 403:252 404:116 412:56 413:252 414:252 415:252 416:65 427:67 428:252 429:252 430:252 431:252 432:20 440:56 441:252 442:252 443:252 444:65 455:67 456:252 457:252 458:252 459:87 460:4 468:56 469:252 470:252 471:124 472:11 483:67 484:252 485:252 486:252 487:54 494:19 495:236 496:245 497:252 498:252 499:98 511:67 512:252 513:252 514:252 515:97 516:5 521:39 522:219 523:252 524:252 525:252 526:252 527:98 539:67 540:252 541:252 542:252 543:252 544:102 545:89 546:89 547:89 548:89 549:203 550:252 551:252 552:252 553:252 554:209 555:64 567:67 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:252 576:252 577:252 578:252 579:252 580:226 581:130 582:68 595:67 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:252 605:252 606:252 607:239 608:77 623:17 624:65 625:163 626:252 627:252 628:252 629:252 630:252 631:252 632:252 633:252 634:96 635:59 653:17 654:176 655:252 656:252 657:252 658:252 659:155 660:32 661:32 662:6 +0 96:56 97:247 98:121 124:24 125:242 126:245 127:122 153:231 154:253 155:253 156:104 157:12 181:90 182:253 183:253 184:254 185:221 186:120 187:120 188:85 206:67 207:75 208:36 209:11 210:56 211:222 212:254 213:253 214:253 215:253 216:245 217:207 218:36 233:86 234:245 235:249 236:105 239:44 240:224 241:230 242:253 243:253 244:253 245:253 246:214 247:10 260:8 261:191 262:253 263:143 269:29 270:119 271:119 272:158 273:253 274:253 275:94 288:15 289:253 290:226 291:48 300:4 301:183 302:253 303:248 304:56 316:42 317:253 318:178 329:179 330:253 331:184 332:14 344:164 345:253 346:178 357:179 358:253 359:163 371:61 372:254 373:254 374:179 384:76 385:254 386:254 387:164 399:60 400:253 401:253 402:178 411:29 412:206 413:253 414:253 415:40 427:60 428:253 429:253 430:178 439:120 440:253 441:253 442:245 443:13 455:60 456:253 457:253 458:178 467:120 468:253 469:239 470:63 483:60 484:253 485:253 486:178 494:14 495:238 496:253 497:179 511:18 512:190 513:253 514:231 515:70 521:43 522:184 523:253 524:253 525:74 540:86 541:253 542:253 543:239 544:134 545:8 548:56 549:163 550:253 551:253 552:213 553:35 568:16 569:253 570:253 571:253 572:253 573:240 574:239 575:239 576:247 577:253 578:253 579:210 580:27 596:4 597:59 598:204 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:250 607:110 626:31 627:122 628:253 629:253 630:253 631:253 632:255 633:217 634:98 +0 125:19 126:164 127:253 128:255 129:253 130:118 131:59 132:36 153:78 154:251 155:251 156:253 157:251 158:251 159:251 160:199 161:45 180:14 181:198 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:204 190:26 208:5 209:117 210:251 211:251 212:243 213:212 214:239 215:251 216:251 217:251 218:218 236:95 237:251 238:251 239:251 240:120 242:175 243:251 244:251 245:251 246:231 263:97 264:237 265:251 266:251 267:251 270:67 271:240 272:251 273:251 274:243 275:108 290:8 291:163 292:251 293:251 294:240 295:81 299:68 300:251 301:251 302:251 303:179 304:9 317:13 318:145 319:251 320:251 321:226 322:80 327:39 328:251 329:251 330:251 331:251 332:115 345:144 346:251 347:251 348:251 349:173 355:18 356:167 357:251 358:251 359:251 360:115 373:233 374:251 375:251 376:251 377:173 384:98 385:251 386:251 387:251 388:115 400:176 401:253 402:253 403:216 404:179 412:99 413:253 414:253 415:253 416:116 427:55 428:210 429:251 430:251 431:96 440:98 441:251 442:251 443:214 444:62 455:117 456:251 457:251 458:251 459:96 467:28 468:204 469:251 470:237 471:53 482:55 483:241 484:251 485:251 486:160 487:7 494:28 495:222 496:251 497:251 498:231 510:59 511:251 512:251 513:251 514:153 520:23 521:98 522:204 523:251 524:251 525:251 526:156 538:59 539:251 540:251 541:251 542:153 546:85 547:155 548:179 549:251 550:251 551:251 552:251 553:154 554:15 566:59 567:251 568:251 569:251 570:236 571:214 572:214 573:214 574:234 575:251 576:253 577:251 578:251 579:248 580:156 581:15 594:41 595:209 596:251 597:251 598:251 599:251 600:251 601:251 602:251 603:251 604:253 605:251 606:196 607:146 623:54 624:115 625:241 626:251 627:251 628:251 629:251 630:251 631:251 632:253 633:187 634:35 653:83 654:251 655:251 656:251 657:251 658:251 659:101 660:57 661:31 +1 129:232 130:255 131:107 156:58 157:244 158:253 159:106 184:95 185:253 186:253 187:106 212:95 213:253 214:253 215:106 240:95 241:253 242:249 243:69 268:144 269:253 270:192 295:97 296:233 297:253 298:66 323:195 324:253 325:253 326:5 350:38 351:232 352:253 353:182 354:2 377:10 378:160 379:253 380:231 381:53 405:42 406:253 407:253 408:158 433:141 434:253 435:253 436:115 460:75 461:245 462:253 463:183 464:4 487:1 488:147 489:253 490:251 491:58 515:20 516:253 517:253 518:180 543:202 544:253 545:226 546:27 571:243 572:253 573:212 598:85 599:251 600:253 601:173 626:209 627:253 628:244 629:57 654:169 655:253 656:174 +1 127:63 128:128 129:2 155:63 156:254 157:123 183:63 184:254 185:179 211:63 212:254 213:179 239:63 240:254 241:179 267:142 268:254 269:179 295:187 296:254 297:158 323:187 324:254 325:55 350:68 351:235 352:254 353:55 378:181 379:254 380:254 381:55 406:181 407:254 408:202 409:14 434:181 435:254 436:186 462:181 463:254 464:146 490:181 491:254 492:62 518:181 519:254 520:62 546:181 547:254 548:62 574:181 575:255 576:62 602:181 603:254 604:241 605:52 630:181 631:254 632:222 633:30 658:181 659:224 660:34 +1 130:131 131:255 132:184 133:15 157:99 158:247 159:253 160:182 161:15 185:124 186:253 187:253 188:253 189:38 212:9 213:171 214:253 215:253 216:140 217:1 240:47 241:253 242:253 243:251 244:117 267:43 268:219 269:253 270:253 271:153 295:78 296:253 297:253 298:253 299:84 323:97 324:253 325:253 326:244 327:74 350:69 351:243 352:253 353:253 354:183 377:10 378:168 379:253 380:253 381:215 382:34 405:31 406:253 407:253 408:253 409:129 433:107 434:253 435:253 436:242 437:67 460:24 461:204 462:253 463:253 464:187 488:95 489:253 490:253 491:201 492:25 516:239 517:253 518:253 519:176 543:119 544:251 545:253 546:253 547:138 570:30 571:212 572:253 573:252 574:165 575:8 598:193 599:253 600:253 601:222 626:193 627:253 628:253 629:189 654:193 655:253 656:201 657:27 +0 125:57 126:255 127:253 128:198 129:85 153:168 154:253 155:251 156:253 157:251 158:169 159:56 180:86 181:253 182:254 183:253 184:254 185:253 186:254 187:253 188:57 208:197 209:251 210:253 211:251 212:253 213:251 214:253 215:251 216:225 217:56 235:169 236:255 237:253 238:226 239:56 241:114 242:254 243:253 244:254 245:84 262:57 263:224 264:253 265:251 266:56 270:139 271:251 272:253 273:83 290:141 291:253 292:255 293:84 298:57 299:225 300:254 301:196 318:253 319:251 320:253 321:83 327:168 328:253 329:83 345:169 346:254 347:253 348:169 355:169 356:254 357:253 358:169 373:168 374:253 375:251 376:56 383:168 384:253 385:251 386:56 401:169 402:254 403:84 412:254 413:253 429:168 430:253 431:83 440:253 441:251 456:29 457:197 458:254 459:84 467:169 468:254 469:196 484:85 485:251 486:253 487:83 494:57 495:224 496:253 497:83 512:57 513:225 514:254 515:139 521:57 522:141 523:253 524:254 525:84 541:168 542:253 543:251 544:169 545:56 547:114 548:169 549:224 550:253 551:251 552:253 553:83 569:169 570:254 571:253 572:254 573:253 574:254 575:253 576:254 577:253 578:254 579:253 580:226 581:56 597:56 598:253 599:251 600:253 601:251 602:253 603:251 604:253 605:251 606:253 607:251 608:56 626:169 627:225 628:254 629:253 630:254 631:253 632:254 633:253 634:226 635:56 655:56 656:253 657:251 658:253 659:251 660:84 661:83 662:56 +0 127:12 128:105 129:224 130:255 131:247 132:22 155:131 156:254 157:254 158:243 159:252 160:76 182:131 183:225 184:254 185:224 186:48 187:136 208:13 209:109 210:252 211:254 212:254 213:254 214:197 215:76 235:9 236:181 237:254 238:254 239:240 240:229 241:237 242:254 243:252 244:152 245:21 262:9 263:143 264:254 265:254 266:226 267:36 269:22 270:138 271:254 272:254 273:188 289:13 290:181 291:254 292:254 293:250 294:64 298:2 299:53 300:236 301:252 302:131 317:102 318:254 319:254 320:254 321:111 328:56 329:243 330:251 331:42 344:30 345:186 346:254 347:254 348:206 349:29 357:199 358:254 359:91 372:92 373:254 374:254 375:237 376:13 385:134 386:254 387:91 400:133 401:254 402:254 403:126 413:134 414:250 415:17 428:187 429:254 430:237 431:23 441:200 442:183 456:187 457:254 458:213 467:2 468:134 469:252 470:101 484:183 485:254 486:133 495:14 496:254 497:234 498:34 512:92 513:254 514:161 522:84 523:204 524:254 525:56 540:92 541:254 542:229 549:85 550:252 551:252 552:188 553:11 568:56 569:252 570:229 575:3 576:53 577:235 578:253 579:166 597:224 598:245 599:130 600:68 601:68 602:134 603:214 604:254 605:254 606:159 625:141 626:254 627:254 628:254 629:254 630:254 631:254 632:233 633:95 634:3 653:14 654:152 655:254 656:254 657:254 658:186 659:157 660:53 +1 130:226 131:247 132:55 157:99 158:248 159:254 160:230 161:30 185:125 186:254 187:254 188:254 189:38 213:125 214:254 215:254 216:212 217:24 240:18 241:223 242:254 243:252 244:118 268:24 269:254 270:254 271:239 295:27 296:195 297:254 298:254 299:93 323:78 324:254 325:254 326:246 327:74 351:158 352:254 353:254 354:185 378:41 379:239 380:254 381:254 382:43 405:22 406:218 407:254 408:254 409:167 410:9 433:32 434:254 435:254 436:254 437:130 460:24 461:187 462:254 463:254 464:234 465:16 488:189 489:254 490:254 491:254 492:128 515:64 516:247 517:254 518:255 519:219 520:42 543:139 544:254 545:254 546:222 547:40 570:30 571:213 572:254 573:235 574:45 598:194 599:254 600:254 601:223 626:194 627:254 628:254 629:190 654:194 655:254 656:202 657:27 +1 130:166 131:253 132:124 133:53 158:140 159:251 160:251 161:180 185:125 186:246 187:251 188:251 189:51 212:32 213:190 214:251 215:251 216:251 217:103 240:21 241:174 242:251 243:251 244:251 268:73 269:176 270:253 271:253 272:201 296:149 297:251 298:251 299:251 300:71 323:27 324:228 325:251 326:251 327:157 328:10 351:180 352:253 353:251 354:251 355:142 377:27 378:180 379:231 380:253 381:251 382:96 383:41 405:89 406:253 407:253 408:255 409:211 410:25 433:217 434:251 435:251 436:253 437:107 460:21 461:221 462:251 463:251 464:242 465:92 487:32 488:190 489:251 490:251 491:251 492:103 515:202 516:251 517:251 518:251 519:122 542:53 543:255 544:253 545:253 546:221 547:51 570:180 571:253 572:251 573:251 574:142 598:180 599:253 600:251 601:251 602:142 626:180 627:253 628:251 629:157 630:82 654:180 655:253 656:147 657:10 +1 129:17 130:206 131:229 132:44 157:2 158:125 159:254 160:123 185:95 186:254 187:254 188:123 212:78 213:240 214:254 215:254 216:123 240:100 241:254 242:254 243:254 244:123 267:2 268:129 269:254 270:254 271:220 272:20 295:9 296:254 297:254 298:254 299:123 322:22 323:179 324:254 325:254 326:254 327:49 350:83 351:254 352:254 353:254 354:183 355:19 378:136 379:254 380:254 381:254 382:139 404:3 405:111 406:252 407:254 408:254 409:232 410:45 432:67 433:254 434:254 435:254 436:216 437:40 459:14 460:192 461:254 462:254 463:254 464:140 486:23 487:192 488:254 489:254 490:254 491:246 514:77 515:254 516:254 517:255 518:241 519:100 541:65 542:235 543:254 544:254 545:254 546:172 568:30 569:238 570:254 571:254 572:254 573:219 574:26 596:34 597:254 598:254 599:254 600:216 601:41 624:34 625:254 626:254 627:254 628:188 652:12 653:170 654:254 655:254 656:82 +1 130:218 131:253 132:124 157:84 158:236 159:251 160:251 184:63 185:236 186:251 187:251 188:122 212:73 213:251 214:251 215:251 216:173 240:202 241:251 242:251 243:251 244:71 267:53 268:255 269:253 270:253 271:253 272:72 295:180 296:253 297:251 298:251 299:188 300:30 323:180 324:253 325:251 326:251 327:142 350:47 351:211 352:253 353:251 354:235 355:82 377:27 378:211 379:251 380:253 381:251 382:215 405:89 406:253 407:253 408:255 409:253 410:164 433:217 434:251 435:251 436:253 437:168 438:15 460:21 461:221 462:251 463:251 464:253 465:107 487:32 488:190 489:251 490:251 491:251 492:221 493:61 515:73 516:251 517:251 518:251 519:251 520:180 543:255 544:253 545:253 546:253 547:201 570:105 571:253 572:251 573:251 574:251 575:71 598:180 599:253 600:251 601:246 602:137 603:10 626:180 627:253 628:251 629:215 654:180 655:253 656:251 657:86 +1 124:102 125:180 126:1 152:140 153:254 154:130 180:140 181:254 182:204 208:140 209:254 210:204 236:72 237:254 238:204 264:25 265:231 266:250 267:135 292:11 293:211 294:254 295:222 321:101 322:254 323:250 324:15 349:96 350:254 351:254 352:95 377:2 378:251 379:254 380:95 405:2 406:251 407:254 408:95 433:96 434:254 435:254 436:95 461:53 462:253 463:254 464:139 490:250 491:254 492:235 493:27 518:201 519:254 520:254 521:128 546:80 547:254 548:254 549:139 574:65 575:254 576:254 577:139 602:150 603:254 604:254 605:139 630:229 631:254 632:254 633:43 658:52 659:196 660:168 661:9 +0 128:87 129:208 130:249 155:27 156:212 157:254 158:195 182:118 183:225 184:254 185:254 186:232 187:147 188:46 209:115 210:248 211:254 212:254 213:254 214:254 215:254 216:230 217:148 218:12 236:18 237:250 238:254 239:245 240:226 241:254 242:254 243:254 244:254 245:254 246:148 263:92 264:205 265:254 266:250 267:101 268:20 269:194 270:254 271:254 272:254 273:254 274:229 275:53 291:152 292:254 293:254 294:94 297:14 298:124 299:187 300:254 301:254 302:254 303:213 318:95 319:252 320:254 321:206 322:15 327:3 328:6 329:51 330:231 331:254 332:94 345:50 346:246 347:254 348:254 349:20 358:200 359:254 360:96 372:21 373:184 374:254 375:254 376:147 377:2 386:200 387:254 388:96 400:177 401:254 402:254 403:218 404:33 413:16 414:211 415:254 416:96 427:11 428:219 429:254 430:251 431:92 441:84 442:254 443:232 444:44 455:101 456:254 457:254 458:141 469:162 470:254 471:231 472:42 483:235 484:254 485:227 486:42 496:51 497:238 498:254 499:213 511:235 512:254 513:199 524:160 525:254 526:229 527:52 539:235 540:254 541:199 549:10 550:84 551:150 552:253 553:254 554:147 567:235 568:254 569:213 570:20 575:17 576:63 577:158 578:254 579:254 580:254 581:155 582:12 595:122 596:248 597:254 598:204 599:98 600:42 601:177 602:180 603:200 604:254 605:254 606:253 607:213 608:82 609:10 624:203 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:251 633:219 634:94 652:35 653:221 654:254 655:254 656:254 657:254 658:254 659:217 660:95 +1 126:134 127:230 154:133 155:231 156:10 182:133 183:253 184:96 210:133 211:253 212:96 238:133 239:253 240:183 266:133 267:253 268:217 294:133 295:253 296:217 322:133 323:253 324:217 350:133 351:253 352:217 378:133 379:253 380:217 406:134 407:254 408:218 434:133 435:253 436:159 462:133 463:253 464:199 490:156 491:253 492:96 518:254 519:247 520:73 546:254 547:248 548:74 573:99 574:254 575:245 576:64 600:89 601:230 602:254 603:125 627:140 628:251 629:253 630:243 631:10 655:114 656:242 657:195 658:69 +1 125:29 126:85 127:255 128:139 153:197 154:251 155:253 156:251 181:254 182:253 183:254 184:253 209:253 210:251 211:253 212:251 237:254 238:253 239:254 240:253 265:253 266:251 267:253 268:138 293:254 294:253 295:254 296:196 321:253 322:251 323:253 324:196 349:254 350:253 351:254 352:84 377:253 378:251 379:253 380:196 405:254 406:253 407:254 408:253 433:253 434:251 435:253 436:251 461:254 462:253 463:254 464:253 489:253 490:251 491:253 492:251 517:254 518:253 519:254 520:253 545:253 546:251 547:253 548:251 573:254 574:253 575:254 576:253 601:253 602:251 603:253 604:251 629:57 630:225 631:254 632:253 658:56 659:253 660:251 +1 125:149 126:255 127:254 128:58 153:215 154:253 155:183 156:2 180:41 181:232 182:253 183:181 208:92 209:253 210:253 211:181 236:92 237:253 238:253 239:181 264:92 265:253 266:253 267:181 292:92 293:253 294:253 295:181 320:92 321:253 322:253 323:181 348:92 349:253 350:253 351:181 376:92 377:253 378:253 379:181 404:92 405:253 406:253 407:181 432:92 433:253 434:253 435:181 460:92 461:253 462:253 463:181 488:31 489:228 490:253 491:181 517:198 518:253 519:228 520:54 545:33 546:226 547:253 548:195 549:7 574:199 575:253 576:253 577:75 602:34 603:218 604:253 605:228 606:117 607:14 608:12 631:33 632:219 633:253 634:253 635:253 636:211 660:32 661:123 662:149 663:230 664:41 +1 130:79 131:203 132:141 157:51 158:240 159:240 160:140 185:88 186:252 187:252 188:140 213:197 214:252 215:252 216:140 241:197 242:252 243:252 244:140 268:147 269:253 270:253 271:253 295:38 296:234 297:252 298:242 299:89 323:113 324:252 325:252 326:223 350:16 351:207 352:252 353:252 354:129 377:16 378:203 379:253 380:252 381:220 382:37 405:29 406:253 407:255 408:253 409:56 432:19 433:181 434:252 435:253 436:176 437:6 460:166 461:252 462:252 463:228 464:52 487:10 488:203 489:252 490:252 491:126 514:63 515:178 516:252 517:252 518:173 542:114 543:253 544:253 545:225 570:238 571:252 572:252 573:99 596:7 597:135 598:253 599:252 600:176 601:19 624:29 625:252 626:253 627:252 628:55 652:13 653:189 654:253 655:204 656:25 +1 126:94 127:254 128:75 154:166 155:253 156:231 182:208 183:253 184:147 210:208 211:253 212:116 238:208 239:253 240:168 266:146 267:254 268:222 294:166 295:253 296:116 322:208 323:253 324:116 350:166 351:253 352:158 378:145 379:253 380:231 406:209 407:254 408:169 434:187 435:253 436:168 462:93 463:253 464:116 490:93 491:253 492:116 518:93 519:253 520:116 546:94 547:254 548:179 549:11 574:93 575:253 576:246 577:101 602:145 603:253 604:255 605:92 630:93 631:253 632:246 633:59 658:93 659:253 660:74 +0 127:46 128:105 129:254 130:254 131:224 132:59 133:59 134:9 155:196 156:254 157:253 158:253 159:253 160:253 161:253 162:128 182:96 183:235 184:254 185:253 186:253 187:253 188:253 189:253 190:247 191:122 208:4 209:101 210:244 211:253 212:254 213:234 214:241 215:253 216:253 217:253 218:253 219:186 220:18 236:96 237:253 238:253 239:253 240:232 241:83 242:109 243:170 244:253 245:253 246:253 247:253 248:116 264:215 265:253 266:253 267:253 268:196 271:40 272:253 273:253 274:253 275:253 276:116 290:8 291:141 292:247 293:253 294:253 295:237 296:29 299:6 300:38 301:171 302:253 303:253 304:116 317:13 318:146 319:253 320:253 321:253 322:253 323:57 329:156 330:253 331:253 332:116 345:40 346:253 347:253 348:253 349:253 350:178 351:27 357:156 358:253 359:253 360:116 372:136 373:204 374:253 375:253 376:253 377:192 378:27 385:156 386:253 387:253 388:116 399:28 400:195 401:254 402:254 403:254 404:250 405:135 412:99 413:255 414:254 415:254 416:117 427:118 428:253 429:253 430:253 431:253 432:142 439:19 440:170 441:253 442:253 443:216 444:62 454:42 455:212 456:253 457:253 458:253 459:253 460:38 466:124 467:188 468:253 469:253 470:253 471:174 482:59 483:253 484:253 485:253 486:237 487:93 488:3 491:31 492:40 493:130 494:247 495:253 496:253 497:253 498:204 499:13 510:59 511:253 512:253 513:253 514:154 518:54 519:218 520:254 521:253 522:253 523:253 524:253 525:253 526:38 538:59 539:253 540:253 541:253 542:215 543:156 544:156 545:156 546:209 547:253 548:255 549:253 550:253 551:253 552:192 553:97 554:15 566:55 567:242 568:253 569:253 570:253 571:253 572:253 573:253 574:253 575:253 576:254 577:253 578:253 579:204 580:23 595:118 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:216 606:174 607:13 623:54 624:116 625:243 626:253 627:253 628:253 629:253 630:253 631:146 632:117 633:62 653:53 654:132 655:253 656:253 657:192 658:57 659:13 +1 125:42 126:232 127:254 128:58 153:86 154:253 155:253 156:58 181:86 182:253 183:253 184:58 209:206 210:253 211:253 212:58 237:215 238:253 239:253 240:58 265:215 266:253 267:253 268:58 293:215 294:253 295:253 296:58 321:215 322:253 323:253 324:58 349:215 350:253 351:253 352:58 377:215 378:253 379:253 380:58 405:215 406:253 407:253 408:58 433:188 434:253 435:253 436:85 461:86 462:253 463:253 464:200 465:12 489:29 490:223 491:253 492:253 493:151 518:209 519:253 520:253 521:194 546:128 547:253 548:253 549:200 550:8 574:32 575:213 576:253 577:253 578:152 579:6 603:32 604:221 605:253 606:253 607:153 608:5 632:90 633:215 634:253 635:253 636:151 661:59 662:253 663:253 664:84 +1 156:60 157:229 158:38 184:187 185:254 186:78 211:121 212:252 213:254 214:78 239:197 240:254 241:206 242:6 267:197 268:254 269:202 294:27 295:218 296:233 297:62 322:117 323:254 324:195 350:203 351:254 352:195 377:64 378:244 379:254 380:195 405:79 406:254 407:255 408:161 433:79 434:254 435:254 436:65 461:79 462:254 463:241 464:52 489:79 490:254 491:189 517:79 518:254 519:189 545:79 546:254 547:189 573:79 574:254 575:189 601:79 602:254 603:194 604:5 629:35 630:219 631:254 632:72 658:34 659:223 660:195 687:129 688:195 +1 101:11 102:150 103:72 129:37 130:251 131:71 157:63 158:251 159:71 185:217 186:251 187:71 213:217 214:251 215:71 240:145 241:253 242:253 243:72 267:42 268:206 269:251 270:251 271:71 295:99 296:251 297:251 298:251 299:71 323:253 324:251 325:251 326:251 327:71 350:130 351:253 352:251 353:251 354:251 355:71 377:110 378:253 379:255 380:253 381:253 382:253 383:72 405:109 406:251 407:253 408:251 409:251 410:188 411:30 433:109 434:251 435:253 436:251 437:246 438:123 460:16 461:170 462:251 463:253 464:251 465:215 488:37 489:251 490:251 491:253 492:251 493:86 516:218 517:253 518:253 519:255 520:253 521:35 543:84 544:236 545:251 546:251 547:253 548:168 549:15 571:144 572:251 573:251 574:251 575:190 576:15 599:144 600:251 601:251 602:251 603:180 626:53 627:221 628:251 629:251 630:251 631:180 +0 127:45 128:254 129:254 130:254 131:148 132:24 133:9 154:43 155:254 156:252 157:252 158:252 159:252 160:252 161:121 162:13 181:58 182:237 183:254 184:252 185:252 186:252 187:252 188:252 189:252 190:68 208:69 209:224 210:252 211:254 212:252 213:252 214:252 215:252 216:252 217:252 218:135 219:17 235:75 236:216 237:252 238:252 239:254 240:231 241:168 242:252 243:252 244:252 245:252 246:252 247:45 262:77 263:212 264:252 265:252 266:252 267:242 268:93 269:32 270:114 271:177 272:252 273:252 274:252 275:158 276:12 289:75 290:212 291:252 292:252 293:252 294:252 295:231 299:116 300:252 301:252 302:252 303:252 304:21 316:69 317:216 318:252 319:252 320:252 321:252 322:252 323:62 327:116 328:252 329:252 330:252 331:252 332:21 344:93 345:252 346:252 347:252 348:252 349:252 350:62 355:21 356:158 357:252 358:252 359:252 360:21 371:64 372:239 373:252 374:252 375:252 376:252 377:252 378:21 384:139 385:252 386:252 387:252 388:21 398:5 399:87 400:254 401:254 402:254 403:254 404:237 405:41 411:11 412:150 413:254 414:254 415:254 416:22 425:5 426:85 427:252 428:252 429:252 430:252 431:222 432:55 439:116 440:252 441:252 442:252 443:214 444:18 453:24 454:252 455:252 456:252 457:252 458:252 459:91 466:26 467:153 468:252 469:252 470:252 471:45 481:24 482:252 483:252 484:252 485:252 486:252 487:91 492:18 493:93 494:151 495:252 496:252 497:252 498:184 499:28 509:24 510:252 511:252 512:252 513:252 514:252 515:164 516:116 517:116 518:116 519:117 520:141 521:252 522:252 523:252 524:252 525:252 526:68 537:24 538:252 539:252 540:252 541:252 542:252 543:252 544:252 545:252 546:252 547:254 548:252 549:252 550:252 551:252 552:252 553:163 554:31 565:9 566:121 567:252 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:254 576:252 577:252 578:252 579:178 580:91 581:33 594:13 595:119 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:254 604:252 605:252 606:184 607:37 623:13 624:121 625:252 626:252 627:252 628:252 629:252 630:252 631:254 632:214 633:45 634:28 652:8 653:21 654:21 655:169 656:252 657:252 658:41 659:22 660:18 +0 125:218 126:253 127:253 128:255 129:149 130:62 151:42 152:144 153:236 154:251 155:251 156:253 157:251 158:236 159:144 160:144 179:99 180:251 181:251 182:251 183:225 184:253 185:251 186:251 187:251 188:251 189:166 190:16 206:79 207:253 208:251 209:251 210:204 211:41 212:143 213:205 214:251 215:251 216:251 217:253 218:169 219:15 233:79 234:231 235:253 236:251 237:225 238:41 241:41 242:226 243:251 244:251 245:253 246:251 247:164 260:37 261:253 262:253 263:255 264:253 265:35 271:79 272:232 273:255 274:253 275:227 276:42 288:140 289:251 290:251 291:253 292:168 293:15 300:77 301:253 302:251 303:251 304:142 315:21 316:221 317:251 318:251 319:164 320:15 329:227 330:251 331:251 332:236 333:61 342:32 343:190 344:251 345:251 346:251 357:73 358:251 359:251 360:251 361:71 370:73 371:251 372:251 373:251 374:251 385:73 386:251 387:251 388:251 389:71 398:73 399:253 400:253 401:253 402:201 413:73 414:253 415:253 416:253 417:72 426:176 427:251 428:251 429:251 430:71 441:73 442:251 443:251 444:251 445:71 454:253 455:251 456:251 457:157 458:10 469:73 470:251 471:251 472:251 473:71 482:253 483:251 484:251 485:142 497:150 498:251 499:251 500:204 501:41 510:124 511:251 512:251 513:220 514:180 524:130 525:253 526:251 527:225 528:41 538:73 539:253 540:253 541:253 542:253 543:73 544:73 545:10 549:42 550:73 551:150 552:253 553:255 554:253 555:216 566:31 567:189 568:251 569:251 570:251 571:253 572:251 573:159 574:144 575:144 576:145 577:206 578:251 579:251 580:251 581:253 582:168 583:92 595:20 596:195 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:251 608:225 609:164 610:15 624:21 625:142 626:220 627:253 628:251 629:251 630:251 631:251 632:253 633:251 634:251 635:204 636:41 654:51 655:72 656:174 657:251 658:251 659:251 660:253 661:147 662:71 663:41 +0 127:60 128:96 129:96 130:48 153:16 154:171 155:228 156:253 157:251 158:220 159:51 160:32 181:127 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:80 207:24 208:182 209:236 210:251 211:211 212:189 213:236 214:251 215:251 216:251 217:242 218:193 234:100 235:194 236:251 237:251 238:211 239:35 241:71 242:173 243:251 244:251 245:253 246:240 247:158 248:19 261:64 262:253 263:255 264:253 265:205 266:19 271:40 272:218 273:255 274:253 275:253 276:91 288:16 289:186 290:251 291:253 292:247 293:110 300:39 301:233 302:251 303:251 304:188 315:16 316:189 317:251 318:251 319:205 320:110 329:48 330:220 331:251 332:220 333:48 343:72 344:251 345:251 346:251 347:158 358:51 359:251 360:251 361:232 371:190 372:251 373:251 374:251 375:59 386:32 387:251 388:251 389:251 398:96 399:253 400:253 401:253 402:95 414:32 415:253 416:253 417:193 426:214 427:251 428:251 429:204 430:23 442:52 443:251 444:251 445:94 454:253 455:251 456:251 457:109 469:48 470:221 471:251 472:219 473:47 482:253 483:251 484:251 485:70 497:234 498:251 499:251 500:188 510:253 511:251 512:251 513:188 523:40 524:158 525:253 526:251 527:172 528:70 539:191 540:253 541:253 542:253 543:96 544:24 549:12 550:174 551:253 552:253 553:255 554:221 567:71 568:251 569:251 570:251 571:253 572:205 573:190 574:190 575:190 576:191 577:197 578:251 579:251 580:231 581:221 582:93 595:16 596:126 597:236 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:140 608:47 625:67 626:188 627:189 628:188 629:188 630:188 631:188 632:189 633:188 634:109 635:4 +0 126:32 127:202 128:255 129:253 130:253 131:175 132:21 152:84 153:144 154:190 155:251 156:253 157:251 158:251 159:251 160:174 176:6 177:37 178:166 179:218 180:236 181:251 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:156 204:115 205:251 206:251 207:253 208:251 209:251 210:251 211:251 212:253 213:251 214:251 215:251 216:251 217:180 231:105 232:241 233:251 234:251 235:253 236:251 237:251 238:251 239:122 240:72 241:71 242:71 243:148 244:251 245:180 258:73 259:253 260:253 261:253 262:253 263:202 264:253 265:253 266:143 286:31 287:189 288:251 289:251 290:251 291:31 292:189 293:251 294:142 314:63 315:236 316:251 317:251 318:96 320:124 321:246 322:142 330:21 331:166 332:21 342:73 343:251 344:251 345:251 346:71 349:217 350:142 357:32 358:190 359:251 360:142 370:73 371:251 372:251 373:251 374:71 377:217 378:142 385:73 386:251 387:251 388:142 398:73 399:253 400:253 401:253 402:72 405:156 406:103 413:73 414:253 415:253 416:253 417:72 426:73 427:251 428:251 429:251 430:174 441:73 442:251 443:251 444:251 445:71 454:73 455:251 456:251 457:251 458:251 469:73 470:251 471:251 472:251 473:71 482:42 483:205 484:251 485:251 486:251 487:79 497:73 498:251 499:251 500:251 501:71 511:41 512:226 513:251 514:251 515:232 516:77 525:73 526:251 527:251 528:251 529:71 540:166 541:253 542:253 543:255 544:253 545:227 546:73 547:21 553:125 554:253 555:253 556:143 568:16 569:169 570:251 571:253 572:251 573:251 574:251 575:174 576:105 579:63 580:144 581:253 582:251 583:251 584:142 597:15 598:35 599:253 600:251 601:251 602:251 603:251 604:243 605:217 606:217 607:231 608:251 609:253 610:251 611:220 612:20 627:143 628:142 629:236 630:251 631:251 632:253 633:251 634:251 635:251 636:251 637:253 638:251 639:137 657:61 658:71 659:200 660:253 661:251 662:251 663:251 664:251 665:201 666:71 667:10 +1 130:218 131:170 132:108 157:32 158:227 159:252 160:232 185:129 186:252 187:252 188:252 212:1 213:253 214:252 215:252 216:168 240:144 241:253 242:252 243:236 244:62 268:144 269:253 270:252 271:215 296:144 297:253 298:252 299:112 323:21 324:206 325:253 326:252 327:71 351:99 352:253 353:255 354:119 378:63 379:242 380:252 381:253 382:35 406:94 407:252 408:252 409:154 410:10 433:145 434:237 435:252 436:252 461:255 462:253 463:253 464:108 487:11 488:155 489:253 490:252 491:179 492:15 514:11 515:150 516:252 517:253 518:200 519:20 542:73 543:252 544:252 545:253 546:97 569:47 570:233 571:253 572:253 596:1 597:149 598:252 599:252 600:252 624:1 625:252 626:252 627:246 628:132 652:1 653:169 654:252 655:132 +1 130:116 131:255 132:123 157:29 158:213 159:253 160:122 185:189 186:253 187:253 188:122 213:189 214:253 215:253 216:122 241:189 242:253 243:253 244:122 267:2 268:114 269:243 270:253 271:186 272:19 295:100 296:253 297:253 298:253 299:48 323:172 324:253 325:253 326:253 327:48 351:172 352:253 353:253 354:182 355:19 378:133 379:251 380:253 381:175 382:4 405:107 406:251 407:253 408:253 409:65 432:26 433:194 434:253 435:253 436:214 437:40 459:105 460:205 461:253 462:253 463:125 464:40 487:139 488:253 489:253 490:253 491:81 514:41 515:231 516:253 517:253 518:159 519:16 541:65 542:155 543:253 544:253 545:172 546:4 569:124 570:253 571:253 572:253 573:98 597:124 598:253 599:253 600:214 601:41 624:22 625:207 626:253 627:253 628:139 653:124 654:253 655:162 656:9 diff --git a/mllib/pom.xml b/mllib/pom.xml index e7ce00efc4af6..cdd33dbb7970d 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -31,20 +31,6 @@ Spark Project ML Library http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 4f9eaacf67fe4..780e8bae42b84 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.classification -import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.mllib.util.DataValidators import org.apache.spark.rdd.RDD /** @@ -183,19 +182,4 @@ object LogisticRegressionWithSGD { numIterations: Int): LogisticRegressionModel = { train(input, numIterations, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 4) { - println("Usage: LogisticRegression " + - "") - System.exit(1) - } - val sc = new SparkContext(args(0), "LogisticRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } 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 18658850a2f64..f6f62ce2de04e 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 @@ -20,11 +20,10 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} import org.apache.spark.annotation.Experimental -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** @@ -158,23 +157,4 @@ object NaiveBayes { def train(input: RDD[LabeledPoint], lambda: Double): NaiveBayesModel = { new NaiveBayes(lambda).run(input) } - - def main(args: Array[String]) { - if (args.length != 2 && args.length != 3) { - println("Usage: NaiveBayes []") - System.exit(1) - } - val sc = new SparkContext(args(0), "NaiveBayes") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = if (args.length == 2) { - NaiveBayes.train(data) - } else { - NaiveBayes.train(data, args(2).toDouble) - } - - println("Pi\n: " + model.pi) - println("Theta:\n" + model.theta) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 956654b1fe90a..81b126717e9a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.classification -import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.mllib.util.DataValidators import org.apache.spark.rdd.RDD /** @@ -183,19 +182,4 @@ object SVMWithSGD { def train(input: RDD[LabeledPoint], numIterations: Int): SVMModel = { train(input, numIterations, 1.0, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: SVM ") - System.exit(1) - } - val sc = new SparkContext(args(0), "SVM") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = SVMWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 90cf8525df523..a64c5d44be406 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -21,8 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} -import org.apache.spark.annotation.Experimental -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLUtils @@ -391,33 +390,11 @@ object KMeans { * Returns the squared Euclidean distance between two vectors computed by * [[org.apache.spark.mllib.util.MLUtils#fastSquaredDistance]]. */ - private[clustering] - def fastSquaredDistance(v1: BreezeVectorWithNorm, v2: BreezeVectorWithNorm) - : Double = { + private[clustering] def fastSquaredDistance( + v1: BreezeVectorWithNorm, + v2: BreezeVectorWithNorm): Double = { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } - - @Experimental - def main(args: Array[String]) { - if (args.length < 4) { - println("Usage: KMeans []") - System.exit(1) - } - val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) - val runs = if (args.length >= 5) args(4).toInt else 1 - val sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile) - .map(line => Vectors.dense(line.split(' ').map(_.toDouble))) - .cache() - val model = KMeans.train(data, k, iters, runs) - val cost = model.computeCost(data) - println("Cluster centers:") - for (c <- model.clusterCenters) { - println(" " + c) - } - println("Cost: " + cost) - System.exit(0) - } } /** 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 1f5c746a3457c..2a77e1a9efb2e 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 @@ -21,17 +21,17 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.math.{abs, sqrt} import scala.util.Random import scala.util.Sorting +import scala.util.hashing.byteswap32 -import com.esotericsoftware.kryo.Kryo import org.jblas.{DoubleMatrix, SimpleBlas, Solve} import org.apache.spark.annotation.Experimental import org.apache.spark.broadcast.Broadcast -import org.apache.spark.{Logging, HashPartitioner, Partitioner, SparkContext, SparkConf} +import org.apache.spark.{Logging, HashPartitioner, Partitioner} import org.apache.spark.storage.StorageLevel import org.apache.spark.rdd.RDD -import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils /** * Out-link information for a user or product block. This includes the original user/product IDs @@ -169,34 +169,39 @@ class ALS private ( this.numBlocks } - val partitioner = new HashPartitioner(numBlocks) + val partitioner = new Partitioner { + val numPartitions = numBlocks - val ratingsByUserBlock = ratings.map{ rating => (rating.user % numBlocks, rating) } + def getPartition(x: Any): Int = { + Utils.nonNegativeMod(byteswap32(x.asInstanceOf[Int]), numPartitions) + } + } + + val ratingsByUserBlock = ratings.map{ rating => + (partitioner.getPartition(rating.user), rating) + } val ratingsByProductBlock = ratings.map{ rating => - (rating.product % numBlocks, Rating(rating.product, rating.user, rating.rating)) + (partitioner.getPartition(rating.product), + Rating(rating.product, rating.user, rating.rating)) } - val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) - val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) + val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock, partitioner) + val (productInLinks, productOutLinks) = + makeLinkRDDs(numBlocks, ratingsByProductBlock, partitioner) // Initialize user and product factors randomly, but use a deterministic seed for each // partition so that fault recovery works val seedGen = new Random(seed) val seed1 = seedGen.nextInt() val seed2 = seedGen.nextInt() - // Hash an integer to propagate random bits at all positions, similar to java.util.HashTable - def hash(x: Int): Int = { - val r = x ^ (x >>> 20) ^ (x >>> 12) - r ^ (r >>> 7) ^ (r >>> 4) - } var users = userOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(hash(seed1 ^ index)) + val rand = new Random(byteswap32(seed1 ^ index)) itr.map { case (x, y) => (x, y.elementIds.map(_ => randomFactor(rank, rand))) } } var products = productOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(hash(seed2 ^ index)) + val rand = new Random(byteswap32(seed2 ^ index)) itr.map { case (x, y) => (x, y.elementIds.map(_ => randomFactor(rank, rand))) } @@ -327,13 +332,14 @@ class ALS private ( * Make the out-links table for a block of the users (or products) dataset given the list of * (user, product, rating) values for the users in that block (or the opposite for products). */ - private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating]): OutLinkBlock = { + private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating], + partitioner: Partitioner): OutLinkBlock = { val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) for (r <- ratings) { - shouldSend(userIdToPos(r.user))(r.product % numBlocks) = true + shouldSend(userIdToPos(r.user))(partitioner.getPartition(r.product)) = true } OutLinkBlock(userIds, shouldSend) } @@ -342,14 +348,15 @@ class ALS private ( * Make the in-links table for a block of the users (or products) dataset given a list of * (user, product, rating) values for the users in that block (or the opposite for products). */ - private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating]): InLinkBlock = { + private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating], + partitioner: Partitioner): InLinkBlock = { val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap // Split out our ratings by product block val blockRatings = Array.fill(numBlocks)(new ArrayBuffer[Rating]) for (r <- ratings) { - blockRatings(r.product % numBlocks) += r + blockRatings(partitioner.getPartition(r.product)) += r } val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) for (productBlock <- 0 until numBlocks) { @@ -374,14 +381,14 @@ class ALS private ( * the users (or (blockId, (p, u, r)) for the products). We create these simultaneously to avoid * having to shuffle the (blockId, (u, p, r)) RDD twice, or to cache it. */ - private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)]) + private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)], partitioner: Partitioner) : (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) = { val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { val ratings = elements.map{_._2}.toArray - val inLinkBlock = makeInLinkBlock(numBlocks, ratings) - val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) + val inLinkBlock = makeInLinkBlock(numBlocks, ratings, partitioner) + val outLinkBlock = makeOutLinkBlock(numBlocks, ratings, partitioner) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) }, true) val inLinks = links.mapValues(_._1) @@ -698,45 +705,4 @@ object ALS { : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0) } - - private class ALSRegistrator extends KryoRegistrator { - override def registerClasses(kryo: Kryo) { - kryo.register(classOf[Rating]) - } - } - - def main(args: Array[String]) { - if (args.length < 5 || args.length > 9) { - println("Usage: ALS " + - "[] [] [] []") - System.exit(1) - } - val (master, ratingsFile, rank, iters, outputDir) = - (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) - val lambda = if (args.length >= 6) args(5).toDouble else 0.01 - val implicitPrefs = if (args.length >= 7) args(6).toBoolean else false - val alpha = if (args.length >= 8) args(7).toDouble else 1 - val blocks = if (args.length == 9) args(8).toInt else -1 - val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", classOf[ALSRegistrator].getName) - .set("spark.kryo.referenceTracking", "false") - .set("spark.kryoserializer.buffer.mb", "8") - .set("spark.locality.wait", "10000") - val sc = new SparkContext(master, "ALS", conf) - - val ratings = sc.textFile(ratingsFile).map { line => - val fields = line.split(',') - Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) - } - val model = new ALS(rank = rank, iterations = iters, lambda = lambda, - numBlocks = blocks, implicitPrefs = implicitPrefs, alpha = alpha).run(ratings) - - model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } - .saveAsTextFile(outputDir + "/userFeatures") - model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } - .saveAsTextFile(outputDir + "/productFeatures") - println("Final user/product features written to " + outputDir) - sc.stop() - } } 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 5f0812fd2e0eb..0e6fb1b1caa41 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 @@ -17,10 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** @@ -173,19 +171,4 @@ object LassoWithSGD { numIterations: Int): LassoModel = { train(input, numIterations, 1.0, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: Lasso ") - System.exit(1) - } - val sc = new SparkContext(args(0), "Lasso") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LassoWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 228fa8db3e721..1532ff90d846d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -17,11 +17,9 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.util.MLUtils /** * Regression model trained using LinearRegression. @@ -156,18 +154,4 @@ object LinearRegressionWithSGD { numIterations: Int): LinearRegressionModel = { train(input, numIterations, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: LinearRegression ") - System.exit(1) - } - val sc = new SparkContext(args(0), "LinearRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LinearRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index 027305aa78926..b27e158b43f9a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -21,9 +21,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.annotation.Experimental -/** - * :: Experimental :: - */ @Experimental trait RegressionModel extends Serializable { /** 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 e702027c7c170..5f7e25a9b8be1 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 @@ -17,10 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.linalg.Vector /** @@ -170,21 +168,4 @@ object RidgeRegressionWithSGD { numIterations: Int): RidgeRegressionModel = { train(input, numIterations, 1.0, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: RidgeRegression " + - " ") - System.exit(1) - } - val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, - args(3).toDouble) - - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } 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 3dd410e933fa7..6f1f3883a7e81 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 @@ -17,17 +17,14 @@ package org.apache.spark.mllib.tree -import scala.util.control.Breaks._ - import org.apache.spark.annotation.Experimental -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.SparkContext._ +import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance} +import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom @@ -105,31 +102,34 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo * still survived the filters of the parent nodes. */ - // TODO: Convert for loop to while loop - breakable { - for (level <- 0 until maxDepth) { - - logDebug("#####################################") - logDebug("level = " + level) - logDebug("#####################################") - - // Find best split for all nodes at a level. - val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, - level, filters, splits, bins, maxLevelForSingleGroup) - - for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { - // Extract info for nodes at the current level. - extractNodeInfo(nodeSplitStats, level, index, nodes) - // Extract info for nodes at the next lower level. - extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, - filters) - logDebug("final best split = " + nodeSplitStats._1) - } - require(scala.math.pow(2, level) == splitsStatsForLevel.length) - // Check whether all the nodes at the current level at leaves. - val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) - logDebug("all leaf = " + allLeaf) - if (allLeaf) break // no more tree construction + var level = 0 + var break = false + while (level < maxDepth && !break) { + + logDebug("#####################################") + logDebug("level = " + level) + logDebug("#####################################") + + // Find best split for all nodes at a level. + val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, + level, filters, splits, bins, maxLevelForSingleGroup) + + for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { + // Extract info for nodes at the current level. + extractNodeInfo(nodeSplitStats, level, index, nodes) + // Extract info for nodes at the next lower level. + extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, + filters) + logDebug("final best split = " + nodeSplitStats._1) + } + require(scala.math.pow(2, level) == splitsStatsForLevel.length) + // Check whether all the nodes at the current level at leaves. + val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) + logDebug("all leaf = " + allLeaf) + if (allLeaf) { + break = true // no more tree construction + } else { + level += 1 } } @@ -173,8 +173,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo parentImpurities: Array[Double], filters: Array[List[Filter]]): Unit = { // 0 corresponds to the left child node and 1 corresponds to the right child node. - // TODO: Convert to while loop - for (i <- 0 to 1) { + var i = 0 + while (i <= 1) { // Calculate the index of the node from the node level and the index at the current level. val nodeIndex = scala.math.pow(2, level + 1).toInt - 1 + 2 * index + i if (level < maxDepth - 1) { @@ -193,6 +193,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo logDebug("Filter = " + filter) } } + i += 1 } } } @@ -1100,132 +1101,4 @@ object DecisionTree extends Serializable with Logging { throw new UnsupportedOperationException("approximate histogram not supported yet.") } } - - private val usage = """ - Usage: DecisionTreeRunner [slices] --algo --trainDataDir path --testDataDir path --maxDepth num [--impurity ] [--maxBins num] - """ - - def main(args: Array[String]) { - - if (args.length < 2) { - System.err.println(usage) - System.exit(1) - } - - val sc = new SparkContext(args(0), "DecisionTree") - - val argList = args.toList.drop(1) - type OptionMap = Map[Symbol, Any] - - def nextOption(map : OptionMap, list: List[String]): OptionMap = { - list match { - case Nil => map - case "--algo" :: string :: tail => nextOption(map ++ Map('algo -> string), tail) - case "--impurity" :: string :: tail => nextOption(map ++ Map('impurity -> string), tail) - case "--maxDepth" :: string :: tail => nextOption(map ++ Map('maxDepth -> string), tail) - case "--maxBins" :: string :: tail => nextOption(map ++ Map('maxBins -> string), tail) - case "--trainDataDir" :: string :: tail => nextOption(map ++ Map('trainDataDir -> string) - , tail) - case "--testDataDir" :: string :: tail => nextOption(map ++ Map('testDataDir -> string), - tail) - case string :: Nil => nextOption(map ++ Map('infile -> string), list.tail) - case option :: tail => logError("Unknown option " + option) - sys.exit(1) - } - } - val options = nextOption(Map(), argList) - logDebug(options.toString()) - - // Load training data. - val trainData = loadLabeledData(sc, options.get('trainDataDir).get.toString) - - // Identify the type of algorithm. - val algoStr = options.get('algo).get.toString - val algo = algoStr match { - case "Classification" => Classification - case "Regression" => Regression - } - - // Identify the type of impurity. - val impurityStr = options.getOrElse('impurity, - if (algo == Classification) "Gini" else "Variance").toString - val impurity = impurityStr match { - case "Gini" => Gini - case "Entropy" => Entropy - case "Variance" => Variance - } - - val maxDepth = options.getOrElse('maxDepth, "1").toString.toInt - val maxBins = options.getOrElse('maxBins, "100").toString.toInt - val maxMemUsage = memoryStringToMb(options.getOrElse('maxMemory, "128m").toString) - - val strategy = new Strategy(algo, impurity, maxDepth, maxBins, maxMemory=maxMemUsage) - val model = DecisionTree.train(trainData, strategy) - - - - // Load test data. - val testData = loadLabeledData(sc, options.get('testDataDir).get.toString) - - // Measure algorithm accuracy - if (algo == Classification) { - val accuracy = accuracyScore(model, testData) - logDebug("accuracy = " + accuracy) - } - - if (algo == Regression) { - val mse = meanSquaredError(model, testData) - logDebug("mean square error = " + mse) - } - - sc.stop() - } - - /** - * Load labeled data from a file. The data format used here is - * , ..., - * where , are feature values in Double and is the corresponding label as Double. - * - * @param sc SparkContext - * @param dir Directory to the input data files. - * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is - * the label, and the second element represents the feature values (an array of Double). - */ - private def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { - sc.textFile(dir).map { line => - val parts = line.trim().split(",") - val label = parts(0).toDouble - val features = Vectors.dense(parts.slice(1,parts.length).map(_.toDouble)) - LabeledPoint(label, features) - } - } - - // TODO: Port this method to a generic metrics package. - /** - * Calculates the classifier accuracy. - */ - private def accuracyScore(model: DecisionTreeModel, data: RDD[LabeledPoint], - threshold: Double = 0.5): Double = { - def predictedValue(features: Vector) = { - if (model.predict(features) < threshold) 0.0 else 1.0 - } - val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() - val count = data.count() - logDebug("correct prediction count = " + correctCount) - logDebug("data count = " + count) - correctCount.toDouble / count - } - - // TODO: Port this method to a generic metrics package - /** - * Calculates the mean squared error for regression. - */ - private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { - data.map { y => - val err = tree.predict(y.features) - y.label - err * err - }.mean() - } } 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 5aab9aba8f9c0..4dfcd4b52ec66 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.mllib.util.LocalSparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.Partitioner object ALSSuite { @@ -74,7 +75,6 @@ object ALSSuite { (sampledRatings, trueRatings, truePrefs) } - } @@ -128,6 +128,25 @@ class ALSSuite extends FunSuite with LocalSparkContext { assert(u11 != u2) } + 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) => + Rating(u - 25, p - 25, r) + }) + val correct = data._2 + val model = ALS.train(ratings, 5, 15) + + val pairs = Array.tabulate(50, 50)((u, p) => (u - 25, p - 25)).flatten + val ans = model.predict(sc.parallelize(pairs)).collect() + ans.foreach { r => + val u = r.user + 25 + val p = r.product + 25 + val v = r.rating + val error = v - correct.get(u, p) + assert(math.abs(error) < 0.4) + } + } + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * @@ -140,16 +159,19 @@ class ALSSuite extends FunSuite with LocalSparkContext { * @param implicitPrefs flag to test implicit feedback * @param bulkPredict flag to test bulk prediciton * @param negativeWeights whether the generated data can contain negative values + * @param numBlocks number of blocks to partition users and products into */ def testALS(users: Int, products: Int, features: Int, iterations: Int, samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false, - bulkPredict: Boolean = false, negativeWeights: Boolean = false) + bulkPredict: Boolean = false, negativeWeights: Boolean = false, numBlocks: Int = -1) { val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products, features, samplingRate, implicitPrefs, negativeWeights) val model = implicitPrefs match { - case false => ALS.train(sc.parallelize(sampledRatings), features, iterations) - case true => ALS.trainImplicit(sc.parallelize(sampledRatings), features, iterations) + case false => ALS.train(sc.parallelize(sampledRatings), features, iterations, 0.01, + numBlocks, 0L) + case true => ALS.trainImplicit(sc.parallelize(sampledRatings), features, iterations, 0.01, + numBlocks, 1.0, 0L) } val predictedU = new DoubleMatrix(users, features) diff --git a/pom.xml b/pom.xml index 4ff18afa227be..646753fe30301 100644 --- a/pom.xml +++ b/pom.xml @@ -125,8 +125,9 @@ 1.3.2 1.2.3 8.1.14.v20131031 - 0.3.1 + 0.3.6 3.0.0 + 1.7.4 64m 512m @@ -293,7 +294,7 @@ org.ow2.asm asm-commons - + @@ -308,7 +309,7 @@ org.ow2.asm asm-commons - + @@ -380,20 +381,6 @@ derby 10.4.2.0 - - net.liftweb - lift-json_${scala.binary.version} - 2.5.1 - - - - org.scala-lang - scalap - - - com.codahale.metrics metrics-core @@ -439,6 +426,16 @@ scala-library ${scala.version} + + org.scala-lang + scala-actors + ${scala.version} + + + org.scala-lang + scalap + ${scala.version} + org.scalatest scalatest_${scala.binary.version} @@ -506,7 +503,45 @@ org.apache.avro avro - 1.7.4 + ${avro.version} + + + org.jboss.netty + netty + + + io.netty + netty + + + + + org.apache.avro + avro-ipc + ${avro.version} + + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + org.mortbay.jetty + servlet-api + + + org.apache.velocity + velocity + + + + + org.apache.avro + avro-mapred + ${avro.version} org.jboss.netty @@ -701,7 +736,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + @@ -756,7 +791,7 @@ build-helper-maven-plugin - add-scala-sources + add-scala-sources generate-sources add-source @@ -793,6 +828,17 @@ + + + hadoop-0.23 + + + org.apache.avro + avro + + + + yarn-alpha @@ -804,6 +850,12 @@ yarn + + + org.apache.avro + avro + + @@ -851,8 +903,8 @@ - - + + hadoop-provided @@ -892,10 +944,11 @@ org.apache.zookeeper zookeeper + 3.4.5 provided - + diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 9cb31d70444ff..d540dc0a986e9 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -38,6 +38,7 @@ object MimaBuild { IO.read(excludeFile).split("\n") } + // Exclude a single class and its corresponding object def excludeClass(className: String) = { Seq( excludePackage(className), @@ -48,7 +49,16 @@ object MimaBuild { ProblemFilters.exclude[MissingTypesProblem](className + "$") ) } - def excludeSparkClass(className: String) = excludeClass("org.apache.spark." + className) + + // Exclude a Spark class, that is in the package org.apache.spark + def excludeSparkClass(className: String) = { + excludeClass("org.apache.spark." + className) + } + + // Exclude a Spark package, that is in the package org.apache.spark + def excludeSparkPackage(packageName: String) = { + excludePackage("org.apache.spark." + packageName) + } val packagePrivateExcludes = packagePrivateList.flatMap(excludeClass) @@ -58,10 +68,9 @@ object MimaBuild { SparkBuild.SPARK_VERSION match { case v if v.startsWith("1.0") => Seq( - excludePackage("org.apache.spark.api.java"), - excludePackage("org.apache.spark.streaming.api.java"), - excludePackage("org.apache.spark.streaming.scheduler"), - excludePackage("org.apache.spark.mllib") + excludeSparkPackage("api.java"), + excludeSparkPackage("mllib"), + excludeSparkPackage("streaming") ) ++ excludeSparkClass("rdd.ClassTags") ++ excludeSparkClass("util.XORShiftRandom") ++ @@ -69,14 +78,7 @@ object MimaBuild { excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ excludeSparkClass("mllib.regression.LassoWithSGD") ++ - excludeSparkClass("mllib.regression.LinearRegressionWithSGD") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator#Block") ++ - excludeSparkClass("streaming.dstream.ReportError") ++ - excludeSparkClass("streaming.dstream.ReportBlock") ++ - excludeSparkClass("streaming.dstream.DStream") + excludeSparkClass("mllib.regression.LinearRegressionWithSGD") case _ => Seq() } @@ -87,5 +89,4 @@ object MimaBuild { previousArtifact := None, binaryIssueFilters ++= ignoredABIProblems(sparkHome) ) - } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 33f9d644ca66d..51f7335111166 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -23,6 +23,8 @@ import AssemblyKeys._ import scala.util.Properties import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact +import sbtunidoc.Plugin._ +import UnidocKeys._ import scala.collection.JavaConversions._ @@ -31,6 +33,7 @@ import scala.collection.JavaConversions._ object SparkBuild extends Build { val SPARK_VERSION = "1.0.0-SNAPSHOT" + val SPARK_VERSION_SHORT = SPARK_VERSION.replaceAll("-SNAPSHOT", "") // Hadoop version to build against. For example, "1.0.4" for Apache releases, or // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set @@ -103,7 +106,7 @@ object SparkBuild extends Build { case Some(v) => v.toBoolean } lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" - val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() + val maybeAvro = if (hadoopVersion.startsWith("0.23.")) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() lazy val isHiveEnabled = Properties.envOrNone("SPARK_HIVE") match { case None => DEFAULT_HIVE @@ -184,12 +187,17 @@ object SparkBuild extends Build { // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), // Remove certain packages from Scaladoc - scalacOptions in (Compile,doc) := Seq("-groups", "-skip-packages", Seq( - "akka", - "org.apache.spark.network", - "org.apache.spark.deploy", - "org.apache.spark.util.collection" - ).mkString(":")), + scalacOptions in (Compile, doc) := Seq( + "-groups", + "-skip-packages", Seq( + "akka", + "org.apache.spark.api.python", + "org.apache.spark.network", + "org.apache.spark.deploy", + "org.apache.spark.util.collection" + ).mkString(":"), + "-doc-title", "Spark " + SPARK_VERSION_SHORT + " ScalaDoc" + ), // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), @@ -216,7 +224,7 @@ object SparkBuild extends Build { org.apache apache - 13 + 14 http://spark.apache.org/ @@ -242,7 +250,7 @@ object SparkBuild extends Build { JIRA - https://spark-project.atlassian.net/browse/SPARK + https://issues.apache.org/jira/browse/SPARK ), @@ -283,10 +291,10 @@ object SparkBuild extends Build { publishMavenStyle in MavenCompile := true, publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal), publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn - ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings + ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings ++ genjavadocSettings val akkaVersion = "2.2.3-shaded-protobuf" - val chillVersion = "0.3.1" + val chillVersion = "0.3.6" val codahaleMetricsVersion = "3.0.0" val jblasVersion = "1.2.3" val jettyVersion = "8.1.14.v20131031" @@ -305,6 +313,8 @@ object SparkBuild extends Build { val excludeCurator = ExclusionRule(organization = "org.apache.curator") val excludePowermock = ExclusionRule(organization = "org.powermock") val excludeFastutil = ExclusionRule(organization = "it.unimi.dsi") + val excludeJruby = ExclusionRule(organization = "org.jruby") + val excludeThrift = ExclusionRule(organization = "org.apache.thrift") def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { @@ -344,27 +354,71 @@ object SparkBuild extends Build { "com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm), "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" excludeAll(excludeFastutil), - "org.spark-project" % "pyrolite" % "2.0" + "org.spark-project" % "pyrolite" % "2.0.1" ), libraryDependencies ++= maybeAvro ) - def rootSettings = sharedSettings ++ Seq( - publish := {} + // Create a colon-separate package list adding "org.apache.spark" in front of all of them, + // for easier specification of JavaDoc package groups + def packageList(names: String*): String = { + names.map(s => "org.apache.spark." + s).mkString(":") + } + + def rootSettings = sharedSettings ++ scalaJavaUnidocSettings ++ Seq( + publish := {}, + + unidocProjectFilter in (ScalaUnidoc, unidoc) := + inAnyProject -- inProjects(repl, examples, tools, catalyst, yarn, yarnAlpha), + unidocProjectFilter in (JavaUnidoc, unidoc) := + inAnyProject -- inProjects(repl, examples, bagel, graphx, catalyst, tools, yarn, yarnAlpha), + + // Skip class names containing $ and some internal packages in Javadocs + unidocAllSources in (JavaUnidoc, unidoc) := { + (unidocAllSources in (JavaUnidoc, unidoc)).value + .map(_.filterNot(_.getName.contains("$"))) + .map(_.filterNot(_.getCanonicalPath.contains("akka"))) + .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) + .map(_.filterNot(_.getCanonicalPath.contains("network"))) + .map(_.filterNot(_.getCanonicalPath.contains("executor"))) + .map(_.filterNot(_.getCanonicalPath.contains("python"))) + .map(_.filterNot(_.getCanonicalPath.contains("collection"))) + }, + + // Javadoc options: create a window title, and group key packages on index page + javacOptions in doc := Seq( + "-windowtitle", "Spark " + SPARK_VERSION_SHORT + " JavaDoc", + "-public", + "-group", "Core Java API", packageList("api.java", "api.java.function"), + "-group", "Spark Streaming", packageList( + "streaming.api.java", "streaming.flume", "streaming.kafka", + "streaming.mqtt", "streaming.twitter", "streaming.zeromq" + ), + "-group", "MLlib", packageList( + "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", + "mllib.linalg.distributed", "mllib.optimization", "mllib.rdd", "mllib.recommendation", + "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration", + "mllib.tree.impurity", "mllib.tree.model", "mllib.util" + ), + "-group", "Spark SQL", packageList("sql.api.java", "sql.hive.api.java"), + "-noqualifier", "java.lang" + ) ) def replSettings = sharedSettings ++ Seq( name := "spark-repl", - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ), - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v ), - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v ) + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v), + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v), + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v) ) def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", + jarName in assembly <<= version map { + v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", - "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), + "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), "org.apache.cassandra" % "cassandra-all" % "1.2.6" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") @@ -372,7 +426,8 @@ object SparkBuild extends Build { exclude("io.netty", "netty") exclude("jline","jline") exclude("org.apache.cassandra.deps", "avro") - excludeAll(excludeSLF4J) + excludeAll(excludeSLF4J), + "com.github.scopt" %% "scopt" % "3.2.0" ) ) ++ assemblySettings ++ extraAssemblySettings @@ -407,7 +462,7 @@ object SparkBuild extends Build { def catalystSettings = sharedSettings ++ Seq( name := "catalyst", // The mechanics of rewriting expression ids to compare trees in some test cases makes - // assumptions about the the expression ids being contiguious. Running tests in parallel breaks + // assumptions about the the expression ids being contiguous. Running tests in parallel breaks // this non-deterministically. TODO: FIX THIS. parallelExecution in Test := false, libraryDependencies ++= Seq( @@ -453,7 +508,7 @@ object SparkBuild extends Build { |import org.apache.spark.sql.catalyst.util._ |import org.apache.spark.sql.execution |import org.apache.spark.sql.hive._ - |import org.apache.spark.sql.hive.TestHive._ + |import org.apache.spark.sql.hive.test.TestHive._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin ) @@ -555,7 +610,7 @@ object SparkBuild extends Build { name := "spark-streaming-flume", previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty) + "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeNetty, excludeThrift) ) ) diff --git a/project/plugins.sbt b/project/plugins.sbt index c25a25863d6ed..0cd16fd5bedd4 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -23,3 +23,4 @@ addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.0") +addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0") diff --git a/python/epydoc.conf b/python/epydoc.conf index 081ed215ae60c..b73860bad8263 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -18,8 +18,8 @@ # # Information about the project. -name: PySpark -url: http://spark-project.org +name: Spark 1.0.0 Python API Docs +url: http://spark.apache.org # The list of modules to document. Modules can be named using # dotted names, module filenames, or package directory names. diff --git a/python/pyspark/context.py b/python/pyspark/context.py index f63cc4a55fb98..c74dc5fd4f854 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -429,7 +429,7 @@ def _getJavaStorageLevel(self, storageLevel): storageLevel.deserialized, storageLevel.replication) - def setJobGroup(self, groupId, description): + def setJobGroup(self, groupId, description, interruptOnCancel=False): """ Assigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared. @@ -437,8 +437,41 @@ def setJobGroup(self, groupId, description): Often, a unit of execution in an application consists of multiple Spark actions or jobs. Application programmers can use this method to group all those jobs together and give a group description. Once set, the Spark web UI will associate such jobs with this group. - """ - self._jsc.setJobGroup(groupId, description) + + The application can use L{SparkContext.cancelJobGroup} to cancel all + running jobs in this group. + + >>> import thread, threading + >>> from time import sleep + >>> result = "Not Set" + >>> lock = threading.Lock() + >>> def map_func(x): + ... sleep(100) + ... return x * x + >>> def start_job(x): + ... global result + ... try: + ... sc.setJobGroup("job_to_cancel", "some description") + ... result = sc.parallelize(range(x)).map(map_func).collect() + ... except Exception as e: + ... result = "Cancelled" + ... lock.release() + >>> def stop_job(): + ... sleep(5) + ... sc.cancelJobGroup("job_to_cancel") + >>> supress = lock.acquire() + >>> supress = thread.start_new_thread(start_job, (10,)) + >>> supress = thread.start_new_thread(stop_job, tuple()) + >>> supress = lock.acquire() + >>> print result + Cancelled + + If interruptOnCancel is set to true for the job group, then job cancellation will result + in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + where HDFS may respond to Thread.interrupt() by marking nodes as dead. + """ + self._jsc.setJobGroup(groupId, description, interruptOnCancel) def setLocalProperty(self, key, value): """ @@ -460,6 +493,19 @@ def sparkUser(self): """ return self._jsc.sc().sparkUser() + def cancelJobGroup(self, groupId): + """ + Cancel active jobs for the specified group. See L{SparkContext.setJobGroup} + for more information. + """ + self._jsc.sc().cancelJobGroup(groupId) + + def cancelAllJobs(self): + """ + Cancel all jobs that have been scheduled or are running. + """ + self._jsc.sc().cancelAllJobs() + def _test(): import atexit import doctest diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 3a23e0801fe7b..c5844597c95f2 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -154,7 +154,7 @@ def __init__(self, labels, pi, theta): def predict(self, x): """Return the most likely class for a data vector x""" - return self.labels[numpy.argmax(self.pi + _dot(x, self.theta))] + return self.labels[numpy.argmax(self.pi + _dot(x, self.theta.transpose()))] class NaiveBayes(object): @classmethod diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index d4771d779f9f4..1ee96bb4af37b 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -104,10 +104,10 @@ def test_clustering(self): def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes data = [ - LabeledPoint(0.0, [1, 0]), - LabeledPoint(1.0, [0, 1]), - LabeledPoint(0.0, [2, 0]), - LabeledPoint(1.0, [0, 2]) + LabeledPoint(0.0, [1, 0, 0]), + LabeledPoint(1.0, [0, 1, 1]), + LabeledPoint(0.0, [2, 0, 0]), + LabeledPoint(1.0, [0, 2, 1]) ] rdd = self.sc.parallelize(data) features = [p.features.tolist() for p in data] diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 91fc7e637e2c6..a59778c72130e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -30,6 +30,7 @@ from threading import Thread import warnings import heapq +from random import Random from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long @@ -332,7 +333,7 @@ def distinct(self): .reduceByKey(lambda x, _: x) \ .map(lambda (x, _): x) - def sample(self, withReplacement, fraction, seed): + def sample(self, withReplacement, fraction, seed=None): """ Return a sampled subset of this RDD (relies on numpy and falls back on default random generator if numpy is unavailable). @@ -344,7 +345,7 @@ def sample(self, withReplacement, fraction, seed): return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) # this is ported from scala/spark/RDD.scala - def takeSample(self, withReplacement, num, seed): + def takeSample(self, withReplacement, num, seed=None): """ Return a fixed-size sampled subset of this RDD (currently requires numpy). @@ -381,13 +382,11 @@ def takeSample(self, withReplacement, num, seed): # If the first sample didn't turn out large enough, keep trying to take samples; # this shouldn't happen often because we use a big multiplier for their initial size. # See: scala/spark/RDD.scala + rand = Random(seed) while len(samples) < total: - if seed > sys.maxint - 2: - seed = -1 - seed += 1 - samples = self.sample(withReplacement, fraction, seed).collect() + samples = self.sample(withReplacement, fraction, rand.randint(0, sys.maxint)).collect() - sampler = RDDSampler(withReplacement, fraction, seed+1) + sampler = RDDSampler(withReplacement, fraction, rand.randint(0, sys.maxint)) sampler.shuffle(samples) return samples[0:total] @@ -600,7 +599,7 @@ def _collect_iterator_through_file(self, iterator): def reduce(self, f): """ Reduces the elements of this RDD using the specified commutative and - associative binary operator. + associative binary operator. Currently reduces partitions locally. >>> from operator import add >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(add) @@ -642,7 +641,34 @@ def func(iterator): vals = self.mapPartitions(func).collect() return reduce(op, vals, zeroValue) - # TODO: aggregate + def aggregate(self, zeroValue, seqOp, combOp): + """ + Aggregate the elements of each partition, and then the results for all + the partitions, using a given combine functions and a neutral "zero + value." + + The functions C{op(t1, t2)} is allowed to modify C{t1} and return it + as its result value to avoid object allocation; however, it should not + modify C{t2}. + + The first function (seqOp) can return a different result type, U, than + the type of this RDD. Thus, we need one operation for merging a T into an U + and one operation for merging two U + + >>> seqOp = (lambda x, y: (x[0] + y, x[1] + 1)) + >>> combOp = (lambda x, y: (x[0] + y[0], x[1] + y[1])) + >>> sc.parallelize([1, 2, 3, 4]).aggregate((0, 0), seqOp, combOp) + (10, 4) + >>> sc.parallelize([]).aggregate((0, 0), seqOp, combOp) + (0, 0) + """ + def func(iterator): + acc = zeroValue + for obj in iterator: + acc = seqOp(acc, obj) + yield acc + + return self.mapPartitions(func).fold(zeroValue, combOp) def max(self): diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index aca2ef3b51e98..845a267e311c5 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -19,7 +19,7 @@ import random class RDDSampler(object): - def __init__(self, withReplacement, fraction, seed): + def __init__(self, withReplacement, fraction, seed=None): try: import numpy self._use_numpy = True @@ -27,7 +27,7 @@ def __init__(self, withReplacement, fraction, seed): print >> sys.stderr, "NumPy does not appear to be installed. Falling back to default random generator for sampling." self._use_numpy = False - self._seed = seed + self._seed = seed if seed is not None else random.randint(0, sys.maxint) self._withReplacement = withReplacement self._fraction = fraction self._random = None @@ -38,17 +38,14 @@ def initRandomGenerator(self, split): if self._use_numpy: import numpy self._random = numpy.random.RandomState(self._seed) - for _ in range(0, split): - # discard the next few values in the sequence to have a - # different seed for the different splits - self._random.randint(sys.maxint) else: - import random - random.seed(self._seed) - for _ in range(0, split): - # discard the next few values in the sequence to have a - # different seed for the different splits - random.randint(0, sys.maxint) + self._random = random.Random(self._seed) + + for _ in range(0, split): + # discard the next few values in the sequence to have a + # different seed for the different splits + self._random.randint(0, sys.maxint) + self._split = split self._rand_initialized = True @@ -59,7 +56,7 @@ def getUniformSample(self, split): if self._use_numpy: return self._random.random_sample() else: - return random.uniform(0.0, 1.0) + return self._random.uniform(0.0, 1.0) def getPoissonSample(self, split, mean): if not self._rand_initialized or split != self._split: @@ -73,26 +70,26 @@ def getPoissonSample(self, split, mean): num_arrivals = 1 cur_time = 0.0 - cur_time += random.expovariate(mean) + cur_time += self._random.expovariate(mean) if cur_time > 1.0: return 0 while(cur_time <= 1.0): - cur_time += random.expovariate(mean) + cur_time += self._random.expovariate(mean) num_arrivals += 1 return (num_arrivals - 1) def shuffle(self, vals): - if self._random == None or split != self._split: + if self._random == None: self.initRandomGenerator(0) # this should only ever called on the master so # the split does not matter if self._use_numpy: self._random.shuffle(vals) else: - random.shuffle(vals, self._random) + self._random.shuffle(vals, self._random.random) def func(self, split, iterator): if self._withReplacement: diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 114fa138d0de2..1a62031db5c41 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -143,13 +143,13 @@ def table(self, tableName): """ return SchemaRDD(self._ssql_ctx.table(tableName), self) - def cacheTable(tableName): + def cacheTable(self, tableName): """ Caches the specified table in-memory. """ self._ssql_ctx.cacheTable(tableName) - def uncacheTable(tableName): + def uncacheTable(self, tableName): """ Removes the specified table from the in-memory cache. """ diff --git a/repl/pom.xml b/repl/pom.xml index 78d2fe13c27eb..b761a176ce256 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -31,20 +31,6 @@ Spark Project REPL http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - /usr/share/spark root diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala index dcc139544e2c2..f8432c8af6ed2 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -47,9 +47,13 @@ trait SparkExprTyper extends Logging { var isIncomplete = false reporter.withIncompleteHandler((_, _) => isIncomplete = true) { val trees = codeParser.stmts(line) - if (reporter.hasErrors) Some(Nil) - else if (isIncomplete) None - else Some(trees) + if (reporter.hasErrors) { + Some(Nil) + } else if (isIncomplete) { + None + } else { + Some(trees) + } } } // def parsesAsExpr(line: String) = { @@ -70,8 +74,7 @@ trait SparkExprTyper extends Logging { val sym0 = symbolOfTerm(name) // drop NullaryMethodType val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType) - if (sym.info.typeSymbol eq UnitClass) NoSymbol - else sym + if (sym.info.typeSymbol eq UnitClass) NoSymbol else sym case _ => NoSymbol } } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index beb40e87024bd..296da740687ec 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -963,8 +963,9 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, val master = this.master match { case Some(m) => m case None => { - val prop = System.getenv("MASTER") - if (prop != null) prop else "local[*]" + val envMaster = sys.env.get("MASTER") + val propMaster = sys.props.get("spark.master") + envMaster.orElse(propMaster).getOrElse("local[*]") } } master diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 4155007c6d337..e33f4f9803054 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext +import org.apache.commons.lang3.StringEscapeUtils class ReplSuite extends FunSuite { @@ -185,11 +186,12 @@ class ReplSuite extends FunSuite { out.close() val output = runInterpreter("local", """ - |var file = sc.textFile("%s/input").cache() + |var file = sc.textFile("%s").cache() |file.count() |file.count() |file.count() - """.stripMargin.format(tempDir.getAbsolutePath)) + """.stripMargin.format(StringEscapeUtils.escapeJava( + tempDir.getAbsolutePath + File.separator + "input"))) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Long = 3", output) diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 9d5c6a857bb00..8d2e4baf69e30 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,67 +16,56 @@ ~ limitations under the License. --> - - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../../pom.xml - - + + 4.0.0 + org.apache.spark - spark-catalyst_2.10 - jar - Spark Project Catalyst - http://spark.apache.org/ + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + - - - yarn-alpha - - - org.apache.avro - avro - - - - + org.apache.spark + spark-catalyst_2.10 + jar + Spark Project Catalyst + http://spark.apache.org/ - - - org.scala-lang - scala-reflect - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - com.typesafe - scalalogging-slf4j_${scala.binary.version} - 1.0.1 - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - - + + + org.scala-lang + scala-reflect + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + com.typesafe + scalalogging-slf4j_${scala.binary.version} + 1.0.1 + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 13a19d0adf5e6..8c76a3aa96546 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst import scala.language.implicitConversions import scala.util.parsing.combinator.lexical.StdLexical import scala.util.parsing.combinator.syntactical.StandardTokenParsers +import scala.util.parsing.combinator.PackratParsers import scala.util.parsing.input.CharArrayReader.EofCh import org.apache.spark.sql.catalyst.analysis._ @@ -39,7 +40,7 @@ import org.apache.spark.sql.catalyst.types._ * This is currently included mostly for illustrative purposes. Users wanting more complete support * for a SQL like language should checkout the HiveQL support in the sql/hive sub-project. */ -class SqlParser extends StandardTokenParsers { +class SqlParser extends StandardTokenParsers with PackratParsers { def apply(input: String): LogicalPlan = { phrase(query)(new lexical.Scanner(input)) match { case Success(r, x) => r @@ -152,7 +153,7 @@ class SqlParser extends StandardTokenParsers { lexical.delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":" + ",", ";", "%", "{", "}", ":", "[", "]" ) protected def assignAliases(exprs: Seq[Expression]): Seq[NamedExpression] = { @@ -339,7 +340,10 @@ class SqlParser extends StandardTokenParsers { protected lazy val floatLit: Parser[String] = elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) - protected lazy val baseExpression: Parser[Expression] = + protected lazy val baseExpression: PackratParser[Expression] = + expression ~ "[" ~ expression <~ "]" ^^ { + case base ~ _ ~ ordinal => GetItem(base, ordinal) + } | TRUE ^^^ Literal(true, BooleanType) | FALSE ^^^ Literal(false, BooleanType) | cast | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 1f3fab09e9566..40d2b42a0cda3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -24,7 +24,11 @@ import org.apache.spark.sql.catalyst.types._ /** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { override def foldable = child.foldable - def nullable = child.nullable + def nullable = (child.dataType, dataType) match { + case (StringType, _: NumericType) => true + case (StringType, TimestampType) => true + case _ => child.nullable + } override def toString = s"CAST($child, $dataType)" type EvaluatedType = Any @@ -111,7 +115,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toLong catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1L else 0L) case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t)) case DecimalType => nullOrCast[BigDecimal](_, _.toLong) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) @@ -131,7 +135,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toShort catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1.toShort else 0.toShort) case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toShort) case DecimalType => nullOrCast[BigDecimal](_, _.toShort) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort @@ -141,7 +145,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toByte catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1.toByte else 0.toByte) case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toByte) case DecimalType => nullOrCast[BigDecimal](_, _.toByte) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte @@ -162,7 +166,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toDouble catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1d else 0d) case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t)) case DecimalType => nullOrCast[BigDecimal](_, _.toDouble) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toDouble(b) @@ -172,7 +176,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toFloat catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1f else 0f) case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toFloat) case DecimalType => nullOrCast[BigDecimal](_, _.toFloat) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index da5b2cf5b0362..82c7af684459f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.trees -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.types.{BooleanType, StringType, TimestampType} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.types.BooleanType + object InterpretedPredicate { def apply(expression: Expression): (Row => Boolean) = { @@ -37,10 +38,26 @@ trait Predicate extends Expression { } trait PredicateHelper { - def splitConjunctivePredicates(condition: Expression): Seq[Expression] = condition match { - case And(cond1, cond2) => splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) - case other => other :: Nil + protected def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { + condition match { + case And(cond1, cond2) => + splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) + case other => other :: Nil + } } + + /** + * Returns true if `expr` can be evaluated using only the output of `plan`. This method + * can be used to determine when is is acceptable to move expression evaluation within a query + * plan. + * + * For example consider a join between two relations R(a, b) and S(c, d). + * + * `canEvaluate(Equals(a,b), R)` returns `true` where as `canEvaluate(Equals(a,c), R)` returns + * `false`. + */ + protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = + expr.references.subsetOf(plan.outputSet) } abstract class BinaryPredicate extends BinaryExpression with Predicate { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 6dd816aa91dd1..0e3a8a6bd30a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -19,7 +19,10 @@ package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec +import org.apache.spark.sql.Logging + import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ /** @@ -101,6 +104,55 @@ object PhysicalOperation extends PredicateHelper { } } +/** + * A pattern that finds joins with equality conditions that can be evaluated using hashing + * techniques. For inner joins, any filters on top of the join operator are also matched. + */ +object HashFilteredJoin extends Logging with PredicateHelper { + /** (joinType, rightKeys, leftKeys, condition, leftChild, rightChild) */ + type ReturnType = + (JoinType, Seq[Expression], Seq[Expression], Option[Expression], LogicalPlan, LogicalPlan) + + def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { + // All predicates can be evaluated for inner join (i.e., those that are in the ON + // clause and WHERE clause.) + case FilteredOperation(predicates, join @ Join(left, right, Inner, condition)) => + logger.debug(s"Considering hash inner join on: ${predicates ++ condition}") + splitPredicates(predicates ++ condition, join) + case join @ Join(left, right, joinType, condition) => + logger.debug(s"Considering hash join on: $condition") + splitPredicates(condition.toSeq, join) + case _ => None + } + + // Find equi-join predicates that can be evaluated before the join, and thus can be used + // as join keys. + def splitPredicates(allPredicates: Seq[Expression], join: Join): Option[ReturnType] = { + val Join(left, right, joinType, _) = join + val (joinPredicates, otherPredicates) = allPredicates.partition { + case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || + (canEvaluate(l, right) && canEvaluate(r, left)) => true + case _ => false + } + + val joinKeys = joinPredicates.map { + case Equals(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) + case Equals(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l) + } + + // Do not consider this strategy if there are no join keys. + if (joinKeys.nonEmpty) { + val leftKeys = joinKeys.map(_._1) + val rightKeys = joinKeys.map(_._2) + + Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) + } else { + logger.debug(s"Avoiding hash join with no join keys.") + None + } + } +} + /** * A pattern that collects all adjacent unions and returns their children as a Seq. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 397473e178867..732708e146b04 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -168,7 +168,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { def references = Set.empty } -case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: LogicalPlan) +case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { def output = child.output diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 2cd0d2b0e1385..d287ad73b9e9f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -237,7 +237,26 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("2012-12-11" cast DoubleType, null) checkEvaluation(Literal(123) cast IntegerType, 123) + checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24) + checkEvaluation(Literal(23) + Cast(true, IntegerType), 24) + checkEvaluation(Literal(23f) + Cast(true, FloatType), 24) + checkEvaluation(Literal(BigDecimal(23)) + Cast(true, DecimalType), 24) + checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24) + checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24) + intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} + + assert(("abcdef" cast StringType).nullable === false) + assert(("abcdef" cast BinaryType).nullable === false) + assert(("abcdef" cast BooleanType).nullable === false) + assert(("abcdef" cast TimestampType).nullable === true) + assert(("abcdef" cast LongType).nullable === true) + assert(("abcdef" cast IntegerType).nullable === true) + assert(("abcdef" cast ShortType).nullable === true) + assert(("abcdef" cast ByteType).nullable === true) + assert(("abcdef" cast DecimalType).nullable === true) + assert(("abcdef" cast DoubleType).nullable === true) + assert(("abcdef" cast FloatType).nullable === true) } test("timestamp") { diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 85580ed6b822f..fb3b190b4ec5a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,72 +16,62 @@ ~ limitations under the License. --> - - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../../pom.xml - - + + 4.0.0 + org.apache.spark - spark-sql_2.10 - jar - Spark Project SQL - http://spark.apache.org/ - - - yarn-alpha - - - org.apache.avro - avro - - - - + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-sql_2.10 + jar + Spark Project SQL + http://spark.apache.org/ - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${project.version} - - - com.twitter - parquet-column - ${parquet.version} - - - com.twitter - parquet-hadoop - ${parquet.version} - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - - + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + + + com.twitter + parquet-column + ${parquet.version} + + + com.twitter + parquet-hadoop + ${parquet.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 4d216b5cd14cb..e25201a6c1775 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -249,11 +249,9 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * :: DeveloperApi :: * The primary workflow for executing relational queries using Spark. Designed to allow easy * access to the intermediate phases of query execution for developers. */ - @DeveloperApi protected abstract class QueryExecution { def logical: LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index f2ae5b0fe612f..ca6e0a696405a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import net.razorvine.pickle.Pickler import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} -import org.apache.spark.annotation.{AlphaComponent, Experimental, DeveloperApi} +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ @@ -256,10 +256,11 @@ class SchemaRDD( * @group Query */ @Experimental + override def sample( - fraction: Double, withReplacement: Boolean = true, - seed: Int = (math.random * 1000).toInt) = + fraction: Double, + seed: Long) = new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index a390ab6005dda..3a895e15a4508 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ /** * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java) */ -trait SchemaRDDLike { +private[sql] trait SchemaRDDLike { @transient val sqlContext: SQLContext @transient protected[spark] val logicalPlan: LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 26922f7f336e2..a7347088794a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.util.Utils /** * The entry point for executing Spark SQL queries from a Java program. @@ -84,10 +85,11 @@ class JavaSQLContext(sparkContext: JavaSparkContext) { */ def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): JavaSchemaRDD = { val schema = getSchema(beanClass) - val className = beanClass.getCanonicalName + val className = beanClass.getName val rowRdd = rdd.rdd.mapPartitions { iter => // BeanInfo is not serializable so we must rediscover it remotely for each partition. - val localBeanInfo = Introspector.getBeanInfo(Class.forName(className)) + val localBeanInfo = Introspector.getBeanInfo( + Class.forName(className, true, Utils.getContextOrSparkClassLoader)) val extractors = localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 5be76890afe31..4cd52d8288137 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -200,10 +200,10 @@ private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { } private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { - override def actualSize(v: String): Int = v.getBytes.length + 4 + override def actualSize(v: String): Int = v.getBytes("utf-8").length + 4 override def append(v: String, buffer: ByteBuffer) { - val stringBytes = v.getBytes() + val stringBytes = v.getBytes("utf-8") buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } @@ -211,7 +211,7 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { val length = buffer.getInt() val stringBytes = new Array[Byte](length) buffer.get(stringBytes, 0, length) - new String(stringBytes) + new String(stringBytes, "utf-8") } override def setField(row: MutableRow, ordinal: Int, value: String) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 3a4f071eebedf..36b3b956da96c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -19,12 +19,14 @@ package org.apache.spark.sql.execution import java.util.HashMap +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ /** + * :: DeveloperApi :: * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each * group. * @@ -34,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.physical._ * @param aggregateExpressions expressions that are computed for each group. * @param child the input data source. */ +@DeveloperApi case class Aggregate( partial: Boolean, groupingExpressions: Seq[Expression], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 070557e47c4c7..3b4acb72e87b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.sql.Row @@ -26,6 +27,10 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { override def outputPartitioning = newPartitioning @@ -81,7 +86,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una * [[catalyst.plans.physical.Distribution Distribution]] requirements for each operator by inserting * [[Exchange]] Operators where required. */ -object AddExchange extends Rule[SparkPlan] { +private[sql] object AddExchange extends Rule[SparkPlan] { // TODO: Determine the number of partitions. val numPartitions = 150 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index cff4887936ae1..da1e08be59e23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, Projection} /** + * :: DeveloperApi :: * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional * programming with one important additional feature, which allows the input rows to be joined with @@ -29,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, * @param outer when true, each input row will be output at least once, even if the output of the * given `generator` is empty. `outer` has no effect when `join` is false. */ +@DeveloperApi case class Generate( generator: Generator, join: Boolean, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 5d89697db5f99..50124dd407447 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Logging, Row} import org.apache.spark.sql.catalyst.trees @@ -26,6 +27,10 @@ import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan +/** + * :: DeveloperApi :: + */ +@DeveloperApi abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { self: Product => @@ -51,6 +56,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { } /** + * :: DeveloperApi :: * Allows already planned SparkQueries to be linked into logical query plans. * * Note that in general it is not valid to use this class to link multiple copies of the same @@ -59,6 +65,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { * replace the output attributes with new copies of themselves without breaking any attribute * linking. */ +@DeveloperApi case class SparkLogicalPlan(alreadyPlanned: SparkPlan) extends logical.LogicalPlan with MultiInstanceRelation { @@ -77,15 +84,15 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) } } -trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { +private[sql] trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { self: Product => } -trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] { +private[sql] trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] { self: Product => override def outputPartitioning: Partitioning = child.outputPartitioning } -trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] { +private[sql] trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] { self: Product => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index c30ae5bcc02d0..5067c14ddffeb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -27,7 +27,7 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils -class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { +private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { val kryo = new Kryo() kryo.setRegistrationRequired(false) @@ -50,7 +50,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { } } -object SparkSqlSerializer { +private[sql] object SparkSqlSerializer { // TODO (lian) Using KryoSerializer here is workaround, needs further investigation // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization // related error. @@ -68,7 +68,7 @@ object SparkSqlSerializer { } } -class BigDecimalSerializer extends Serializer[BigDecimal] { +private[sql] class BigDecimalSerializer extends Serializer[BigDecimal] { def write(kryo: Kryo, output: Output, bd: math.BigDecimal) { // TODO: There are probably more efficient representations than strings... output.writeString(bd.toString()) @@ -83,7 +83,7 @@ class BigDecimalSerializer extends Serializer[BigDecimal] { * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize * them as `Array[(k,v)]`. */ -class MapSerializer extends Serializer[Map[_,_]] { +private[sql] class MapSerializer extends Serializer[Map[_,_]] { def write(kryo: Kryo, output: Output, map: Map[_,_]) { kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index fe8bd5a508820..f763106da4e0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -25,54 +25,19 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.parquet._ -abstract class SparkStrategies extends QueryPlanner[SparkPlan] { +private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => - object HashJoin extends Strategy { + object HashJoin extends Strategy with PredicateHelper { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) => - logger.debug(s"Considering join: ${predicates ++ condition}") - // Find equi-join predicates that can be evaluated before the join, and thus can be used - // as join keys. Note we can only mix in the conditions with other predicates because the - // match above ensures that this is and Inner join. - val (joinPredicates, otherPredicates) = (predicates ++ condition).partition { - case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || - (canEvaluate(l, right) && canEvaluate(r, left)) => true - case _ => false - } - - val joinKeys = joinPredicates.map { - case Equals(l,r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) - case Equals(l,r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l) - } - - // Do not consider this strategy if there are no join keys. - if (joinKeys.nonEmpty) { - val leftKeys = joinKeys.map(_._1) - val rightKeys = joinKeys.map(_._2) - - val joinOp = execution.HashJoin( - leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) - - // Make sure other conditions are met if present. - if (otherPredicates.nonEmpty) { - execution.Filter(combineConjunctivePredicates(otherPredicates), joinOp) :: Nil - } else { - joinOp :: Nil - } - } else { - logger.debug(s"Avoiding spark join with no join keys.") - Nil - } + // Find inner joins where at least some predicates can be evaluated by matching hash keys + // using the HashFilteredJoin pattern. + case HashFilteredJoin(Inner, leftKeys, rightKeys, condition, left, right) => + val hashJoin = + execution.HashJoin(leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) + condition.map(Filter(_, hashJoin)).getOrElse(hashJoin) :: Nil case _ => Nil } - - private def combineConjunctivePredicates(predicates: Seq[Expression]) = - predicates.reduceLeft(And) - - /** Returns true if `expr` can be evaluated using only the output of `plan`. */ - protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = - expr.references subsetOf plan.outputSet } object PartialAggregation extends Strategy { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index eedcc7dda02d7..d807187a5ffb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{HashPartitioner, SparkConf, SparkContext} import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.sql.catalyst.ScalaReflection @@ -27,6 +28,10 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{OrderedDistribution, UnspecifiedDistribution} import org.apache.spark.util.MutablePair +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { override def output = projectList.map(_.toAttribute) @@ -36,6 +41,10 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def output = child.output @@ -44,15 +53,23 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { } } -case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan) - extends UnaryNode { - +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: SparkPlan) + extends UnaryNode +{ override def output = child.output // TODO: How to pick seed? override def execute() = child.execute().sample(withReplacement, fraction, seed) } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends SparkPlan { // TODO: attributes output by union should be distinct for nullability purposes override def output = children.head.output @@ -62,12 +79,14 @@ case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends } /** + * :: DeveloperApi :: * Take the first limit elements. Note that the implementation is different depending on whether * this is a terminal operator or not. If it is terminal and is invoked using executeCollect, * this operator uses Spark's take method on the Spark driver. If it is not terminal or is * invoked using execute, we first take the limit on each partition, and then repartition all the * data to a single partition to compute the global limit. */ +@DeveloperApi case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan: // partition local limit -> exchange into one partition -> partition local limit again @@ -91,10 +110,12 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) exte } /** + * :: DeveloperApi :: * Take the first limit elements as defined by the sortOrder. This is logically equivalent to * having a [[Limit]] operator after a [[Sort]] operator. This could have been named TopK, but * Spark's top operator does the opposite in ordering so we name it TakeOrdered to avoid confusion. */ +@DeveloperApi case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) (@transient sc: SparkContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil @@ -111,7 +132,10 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) override def execute() = sc.makeRDD(executeCollect(), 1) } - +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Sort( sortOrder: Seq[SortOrder], global: Boolean, @@ -134,6 +158,10 @@ case class Sort( override def output = child.output } +/** + * :: DeveloperApi :: + */ +@DeveloperApi object ExistingRdd { def convertToCatalyst(a: Any): Any = a match { case s: Seq[Any] => s.map(convertToCatalyst) @@ -167,6 +195,10 @@ object ExistingRdd { } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { override def execute() = rdd } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala index 40982f1fffbbf..a0d29100f505a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -object DebugQuery { +private[sql] object DebugQuery { def apply(plan: SparkPlan): SparkPlan = { val visited = new collection.mutable.HashSet[Long]() plan transform { @@ -28,7 +28,7 @@ object DebugQuery { } } -case class DebugNode(child: SparkPlan) extends UnaryNode { +private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { def references = Set.empty def output = child.output def execute() = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index c89dae9358bf7..31cc26962ad93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -21,14 +21,24 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} +@DeveloperApi sealed abstract class BuildSide + +@DeveloperApi case object BuildLeft extends BuildSide + +@DeveloperApi case object BuildRight extends BuildSide +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class HashJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -130,6 +140,10 @@ case class HashJoin( } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { def output = left.output ++ right.output @@ -138,6 +152,10 @@ case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNod } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class BroadcastNestedLoopJoin( streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression]) (@transient sc: SparkContext) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala index e4a2dec33231d..66237f8f1314b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql /** + * :: DeveloperApi :: * An execution engine for relational query plans that runs on top Spark and returns RDDs. * * Note that the operators in this package are created automatically by a query planner using a diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index d5846baa72ada..f825ca3c028ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -203,8 +203,9 @@ case class InsertIntoParquetTable( val stageId = sc.newRddId() val taskIdOffset = - if (overwrite) 1 - else { + if (overwrite) { + 1 + } else { FileSystemHelper .findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 84b1b4609458b..71ba0fecce47a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -158,8 +158,11 @@ private[parquet] class CatalystGroupConverter( a => a.dataType match { case ctype: NativeType => // note: for some reason matching for StringType fails so use this ugly if instead - if (ctype == StringType) new CatalystPrimitiveStringConverter(this, schema.indexOf(a)) - else new CatalystPrimitiveConverter(this, schema.indexOf(a)) + if (ctype == StringType) { + new CatalystPrimitiveStringConverter(this, schema.indexOf(a)) + } else { + new CatalystPrimitiveConverter(this, schema.indexOf(a)) + } case _ => throw new RuntimeException( s"unable to convert datatype ${a.dataType.toString} in CatalystGroupConverter") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 728e3dd1dc02b..f37976f7313c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -28,7 +28,7 @@ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.util.Utils -object ParquetTestData { +private[sql] object ParquetTestData { val testSchema = """message myrecord { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4c4fd6dbbedb4..dde957d715a28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -28,6 +28,22 @@ class SQLQuerySuite extends QueryTest { // Make sure the tables are loaded. TestData + test("index into array") { + checkAnswer( + sql("SELECT data, data[0], data[0] + data[1], data[0 + 1] FROM arrayData"), + arrayData.map(d => (d.data, d.data(0), d.data(0) + d.data(1), d.data(1))).collect().toSeq) + } + + test("index into array of arrays") { + checkAnswer( + sql( + "SELECT nestedData, nestedData[0][0], nestedData[0][0] + nestedData[0][1] FROM arrayData"), + arrayData.map(d => + (d.nestedData, + d.nestedData(0)(0), + d.nestedData(0)(0) + d.nestedData(0)(1))).collect().toSeq) + } + test("agg") { checkAnswer( sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 271b1d9fcacf8..002b7f0adafab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -66,4 +66,11 @@ object TestData { LowerCaseData(3, "c") :: LowerCaseData(4, "d") :: Nil) lowerCaseData.registerAsTable("lowerCaseData") + + case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) + val arrayData = + TestSQLContext.sparkContext.parallelize( + ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) :: + ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) + arrayData.registerAsTable("arrayData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 1d3608ed2d9ab..325173cf95fdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -58,7 +58,7 @@ class ColumnTypeSuite extends FunSuite { checkActualSize(DOUBLE, Double.MaxValue, 8) checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + 5) + checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) val binary = Array.fill[Byte](4)(0: Byte) checkActualSize(BINARY, binary, 4 + 4) @@ -91,14 +91,16 @@ class ColumnTypeSuite extends FunSuite { testNativeColumnType[StringType.type]( STRING, (buffer: ByteBuffer, string: String) => { - val bytes = string.getBytes() - buffer.putInt(bytes.length).put(string.getBytes) + + val bytes = string.getBytes("utf-8") + buffer.putInt(bytes.length) + buffer.put(bytes) }, (buffer: ByteBuffer) => { val length = buffer.getInt() val bytes = new Array[Byte](length) - buffer.get(bytes, 0, length) - new String(bytes) + buffer.get(bytes) + new String(bytes, "utf-8") }) testColumnType[BinaryType.type, Array[Byte]]( @@ -161,9 +163,13 @@ class ColumnTypeSuite extends FunSuite { buffer.rewind() seq.foreach { expected => + println("buffer = " + buffer + ", expected = " + expected) + val extracted = columnType.extract(buffer) assert( - expected === columnType.extract(buffer), - "Extracted value didn't equal to the original one") + expected === extracted, + "Extracted value didn't equal to the original one. " + + hexDump(expected) + " != " + hexDump(extracted) + + ", buffer = " + dumpBuffer(buffer.duplicate().rewind().asInstanceOf[ByteBuffer])) } } @@ -179,4 +185,28 @@ class ColumnTypeSuite extends FunSuite { } } } + + private def hexDump(value: Any): String = { + if (value.isInstanceOf[String]) { + val sb = new StringBuilder() + for (ch <- value.asInstanceOf[String].toCharArray) { + sb.append(Integer.toHexString(ch & 0xffff)).append(' ') + } + if (! sb.isEmpty) sb.setLength(sb.length - 1) + sb.toString() + } else { + // for now .. + hexDump(value.toString) + } + } + + private def dumpBuffer(buff: ByteBuffer): Any = { + val sb = new StringBuilder() + while (buff.hasRemaining) { + val b = buff.get() + sb.append(Integer.toHexString(b & 0xff)).append(' ') + } + if (! sb.isEmpty) sb.setLength(sb.length - 1) + sb.toString() + } } diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index a662da76ce25a..889d249146b8c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,105 +16,95 @@ ~ limitations under the License. --> - - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../../pom.xml - - + + 4.0.0 + org.apache.spark - spark-hive_2.10 - jar - Spark Project Hive - http://spark.apache.org/ - - - yarn-alpha - - - org.apache.avro - avro - - - - + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-hive_2.10 + jar + Spark Project Hive + http://spark.apache.org/ - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-sql_${scala.binary.version} - ${project.version} - - - org.apache.hive - hive-metastore - ${hive.version} - - - org.apache.hive - hive-exec - ${hive.version} - - - org.codehaus.jackson - jackson-mapper-asl - - - org.apache.hive - hive-serde - ${hive.version} - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + + + org.apache.hive + hive-metastore + ${hive.version} + + + org.apache.hive + hive-exec + ${hive.version} + + + org.codehaus.jackson + jackson-mapper-asl + + + org.apache.hive + hive-serde + ${hive.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + - - - org.apache.maven.plugins - maven-dependency-plugin - 2.4 - - - copy-dependencies - package - - copy-dependencies - - - - ${basedir}/../../lib_managed/jars - false - false - true - org.datanucleus - - - - - - + + + org.apache.maven.plugins + maven-dependency-plugin + 2.4 + + + copy-dependencies + package + + copy-dependencies + + + + ${basedir}/../../lib_managed/jars + false + false + true + org.datanucleus + + + + + + diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 7219c030cb0f0..ab7862f4f9e06 100644 --- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.sql.hive import java.io.IOException import java.text.NumberFormat @@ -28,12 +28,13 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc import org.apache.hadoop.mapred._ import org.apache.hadoop.io.Writable +import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} + /** * Internal helper class that saves an RDD using a Hive OutputFormat. * It is based on [[SparkHadoopWriter]]. */ -protected[spark] -class SparkHiveHadoopWriter( +private[hive] class SparkHiveHadoopWriter( @transient jobConf: JobConf, fileSinkConf: FileSinkDesc) extends Logging @@ -179,7 +180,7 @@ class SparkHiveHadoopWriter( } } -object SparkHiveHadoopWriter { +private[hive] object SparkHiveHadoopWriter { def createPathFromString(path: String, conf: JobConf): Path = { if (path == null) { throw new IllegalArgumentException("Output path is null") 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 c0d8adf43dd07..b21f24dad785d 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 @@ -239,7 +239,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1) /** Extends QueryExecution with hive specific features. */ - @DeveloperApi protected[sql] abstract class QueryExecution extends super.QueryExecution { // TODO: Create mixin for the analyzer instead of overriding things here. override lazy val optimizedPlan = 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 ca75cecf7d885..6c907887db79e 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 @@ -34,12 +34,13 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.hive.execution.{HiveTableScan, InsertIntoHiveTable} import org.apache.spark.sql.columnar.InMemoryColumnarTableScan /* Implicit conversions */ import scala.collection.JavaConversions._ -class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { +private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ val client = Hive.get(hive.hiveconf) @@ -171,7 +172,7 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { override def unregisterAllTables() = {} } -object HiveMetastoreTypes extends RegexParsers { +private[hive] object HiveMetastoreTypes extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = "string" ^^^ StringType | "float" ^^^ FloatType | @@ -229,7 +230,8 @@ object HiveMetastoreTypes extends RegexParsers { } } -case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String]) +private[hive] case class MetastoreRelation + (databaseName: String, tableName: String, alias: Option[String]) (val table: TTable, val partitions: Seq[TPartition]) extends BaseRelation { // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and 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 4dac25b3f60e4..1777e96b6713d 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 @@ -36,20 +36,20 @@ import scala.collection.JavaConversions._ * back for Hive to execute natively. Will be replaced with a native command that contains the * cmd string. */ -case object NativePlaceholder extends Command +private[hive] case object NativePlaceholder extends Command -case class DfsCommand(cmd: String) extends Command +private[hive] case class DfsCommand(cmd: String) extends Command -case class ShellCommand(cmd: String) extends Command +private[hive] case class ShellCommand(cmd: String) extends Command -case class SourceCommand(filePath: String) extends Command +private[hive] case class SourceCommand(filePath: String) extends Command -case class AddJar(jarPath: String) extends Command +private[hive] case class AddJar(jarPath: String) extends Command -case class AddFile(filePath: String) extends Command +private[hive] case class AddFile(filePath: String) extends Command /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ -object HiveQl { +private[hive] object HiveQl { protected val nativeCommands = Seq( "TOK_DESCFUNCTION", "TOK_DESCTABLE", 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 ac817b21a152e..d9a6e0e88932e 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 @@ -23,9 +23,10 @@ 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.execution._ +import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan -trait HiveStrategies { +private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. self: SQLContext#SparkPlanner => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala index 2610100043659..8258ee5fef0eb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala @@ -15,23 +15,27 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.hive.execution import java.io.{BufferedReader, InputStreamReader} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ +import org.apache.spark.sql.hive.HiveContext /* Implicit conversions */ import scala.collection.JavaConversions._ /** + * :: DeveloperApi :: * Transforms the input by forking and running the specified script. * * @param input the set of expression that should be passed to the script. * @param script the command that should be executed. * @param output the attributes that are produced by the script. */ +@DeveloperApi case class ScriptTransformation( input: Seq[Expression], script: String, @@ -67,7 +71,7 @@ case class ScriptTransformation( iter .map(outputProjection) // TODO: Use SerDe - .map(_.mkString("", "\t", "\n").getBytes).foreach(outputStream.write) + .map(_.mkString("", "\t", "\n").getBytes("utf-8")).foreach(outputStream.write) outputStream.close() readerThread.join() outputLines.toIterator 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 b1a26fdabbb85..3ad66a3d7f45f 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 @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} @@ -34,6 +34,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.hive._ /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -99,14 +100,15 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { hiveFilesTemp.delete() hiveFilesTemp.mkdir() - val inRepoTests = if (System.getProperty("user.dir").endsWith("sql/hive")) { - new File("src/test/resources/") + val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { + new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) } else { - new File("sql/hive/src/test/resources") + new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + + File.separator + "resources") } def getHiveFile(path: String): File = { - val stripped = path.replaceAll("""\.\.\/""", "") + val stripped = path.replaceAll("""\.\.\/""", "").replace('/', File.separatorChar) hiveDevHome .map(new File(_, stripped)) .filter(_.exists) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index 821fb22112f87..96faebc5a8687 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.metastore.MetaStoreUtils @@ -30,23 +30,26 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharOb import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ -import org.apache.spark.{SparkHiveHadoopWriter, TaskContext, SparkException} +import org.apache.spark.sql.hive._ +import org.apache.spark.{TaskContext, SparkException} /* Implicits */ import scala.collection.JavaConversions._ /** + * :: DeveloperApi :: * The Hive table scan operator. Column and partition pruning are both handled. * - * @constructor * @param attributes Attributes to be fetched from the Hive table. * @param relation The Hive table be be scanned. * @param partitionPruningPred An optional partition pruning predicate for partitioned table. */ +@DeveloperApi case class HiveTableScan( attributes: Seq[Attribute], relation: MetastoreRelation, @@ -160,6 +163,10 @@ case class HiveTableScan( def output = attributes } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], 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 55a4363af6c76..c7de4ab6d3955 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 @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.types._ /* Implicit conversions */ import scala.collection.JavaConversions._ -object HiveFunctionRegistry +private[hive] object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionFactory with HiveInspectors { def lookupFunction(name: String, children: Seq[Expression]): Expression = { @@ -70,24 +70,26 @@ object HiveFunctionRegistry } def javaClassToDataType(clz: Class[_]): DataType = clz match { + // writable case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType + case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType case c: Class[_] if c == classOf[hadoopIo.Text] => StringType case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType + case c: Class[_] if c == classOf[hadoopIo.BytesWritable] => BinaryType + + // java class case c: Class[_] if c == classOf[java.lang.String] => StringType - case c: Class[_] if c == java.lang.Short.TYPE => ShortType - case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType - case c: Class[_] if c == java.lang.Long.TYPE => LongType - case c: Class[_] if c == java.lang.Double.TYPE => DoubleType - case c: Class[_] if c == java.lang.Byte.TYPE => ByteType - case c: Class[_] if c == java.lang.Float.TYPE => FloatType - case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType + case c: Class[_] if c == classOf[HiveDecimal] => DecimalType + case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType + case c: Class[_] if c == classOf[Array[Byte]] => BinaryType case c: Class[_] if c == classOf[java.lang.Short] => ShortType case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType case c: Class[_] if c == classOf[java.lang.Long] => LongType @@ -95,11 +97,21 @@ object HiveFunctionRegistry case c: Class[_] if c == classOf[java.lang.Byte] => ByteType case c: Class[_] if c == classOf[java.lang.Float] => FloatType case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType + + // primitive type + case c: Class[_] if c == java.lang.Short.TYPE => ShortType + case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType + case c: Class[_] if c == java.lang.Long.TYPE => LongType + case c: Class[_] if c == java.lang.Double.TYPE => DoubleType + case c: Class[_] if c == java.lang.Byte.TYPE => ByteType + case c: Class[_] if c == java.lang.Float.TYPE => FloatType + case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) } } -trait HiveFunctionFactory { +private[hive] trait HiveFunctionFactory { def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) def getFunctionClass(name: String) = getFunctionInfo(name).getFunctionClass def createFunction[UDFType](name: String) = @@ -111,11 +123,19 @@ trait HiveFunctionFactory { case i: hadoopIo.IntWritable => i.get case t: hadoopIo.Text => t.toString case l: hadoopIo.LongWritable => l.get - case d: hadoopIo.DoubleWritable => d.get() + case d: hadoopIo.DoubleWritable => d.get case d: hiveIo.DoubleWritable => d.get case s: hiveIo.ShortWritable => s.get - case b: hadoopIo.BooleanWritable => b.get() + case b: hadoopIo.BooleanWritable => b.get case b: hiveIo.ByteWritable => b.get + case b: hadoopIo.FloatWritable => b.get + case b: hadoopIo.BytesWritable => { + val bytes = new Array[Byte](b.getLength) + System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) + bytes + } + case t: hiveIo.TimestampWritable => t.getTimestamp + case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) case list: java.util.List[_] => list.map(unwrap) case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap case array: Array[_] => array.map(unwrap).toSeq @@ -127,10 +147,13 @@ trait HiveFunctionFactory { case p: java.lang.Byte => p case p: java.lang.Boolean => p case str: String => str + case p: BigDecimal => p + case p: Array[Byte] => p + case p: java.sql.Timestamp => p } } -abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { +private[hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { self: Product => type UDFType @@ -148,7 +171,7 @@ abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory override def toString = s"$nodeName#${functionInfo.getDisplayName}(${children.mkString(",")})" } -case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { +private[hive] case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { import org.apache.spark.sql.hive.HiveFunctionRegistry._ type UDFType = UDF @@ -201,7 +224,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } } -case class HiveGenericUdf(name: String, children: Seq[Expression]) +private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) extends HiveUdf with HiveInspectors { import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ @@ -228,7 +251,7 @@ case class HiveGenericUdf(name: String, children: Seq[Expression]) } } -trait HiveInspectors { +private[hive] trait HiveInspectors { def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) @@ -252,13 +275,17 @@ trait HiveInspectors { /** Converts native catalyst types to the types expected by Hive */ def wrap(a: Any): AnyRef = a match { - case s: String => new hadoopIo.Text(s) + case s: String => new hadoopIo.Text(s) // TODO why should be Text? 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 => b.bigDecimal + case b: Array[Byte] => b + case t: java.sql.Timestamp => t case s: Seq[_] => seqAsJavaList(s.map(wrap)) case m: Map[_,_] => mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) }) @@ -280,6 +307,8 @@ trait HiveInspectors { case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector + case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector } def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { @@ -307,6 +336,14 @@ trait HiveInspectors { case _: JavaShortObjectInspector => ShortType case _: WritableByteObjectInspector => ByteType case _: JavaByteObjectInspector => ByteType + case _: WritableFloatObjectInspector => FloatType + case _: JavaFloatObjectInspector => FloatType + case _: WritableBinaryObjectInspector => BinaryType + case _: JavaBinaryObjectInspector => BinaryType + case _: WritableHiveDecimalObjectInspector => DecimalType + case _: JavaHiveDecimalObjectInspector => DecimalType + case _: WritableTimestampObjectInspector => TimestampType + case _: JavaTimestampObjectInspector => TimestampType } implicit class typeInfoConversions(dt: DataType) { @@ -324,12 +361,13 @@ trait HiveInspectors { case ShortType => shortTypeInfo case StringType => stringTypeInfo case DecimalType => decimalTypeInfo + case TimestampType => timestampTypeInfo case NullType => voidTypeInfo } } } -case class HiveGenericUdaf( +private[hive] case class HiveGenericUdaf( name: String, children: Seq[Expression]) extends AggregateExpression with HiveInspectors @@ -371,7 +409,7 @@ case class HiveGenericUdaf( * Operators that require maintaining state in between input rows should instead be implemented as * user defined aggregations, which have clean semantics even in a partitioned execution. */ -case class HiveGenericUdtf( +private[hive] case class HiveGenericUdtf( name: String, aliasNames: Seq[String], children: Seq[Expression]) @@ -438,7 +476,7 @@ case class HiveGenericUdtf( override def toString = s"$nodeName#$name(${children.mkString(",")})" } -case class HiveUdafFunction( +private[hive] case class HiveUdafFunction( functionName: String, exprs: Seq[Expression], base: AggregateExpression) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 79ec1f1cde019..f9a162ef4e3c0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.columnar.InMemoryColumnarTableScan import org.apache.spark.sql.hive.execution.HiveComparisonTest +import org.apache.spark.sql.hive.test.TestHive class CachedTableSuite extends HiveComparisonTest { TestHive.loadTestTable("src") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index ad29e06905c1b..833f3502154f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql.hive -import java.io.File - import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive /* Implicits */ -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ case class TestData(key: Int, value: String) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala index 8137f99b227f4..9c5d7c81f7c09 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.FunSuite import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.test.TestHive // Implicits import scala.collection.JavaConversions._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 8488f23abd0fd..42a82c1fbf5c7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -19,14 +19,14 @@ package org.apache.spark.sql.hive.execution import java.io.File -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ /** * A set of test cases based on the big-data-benchmark. * https://amplab.cs.berkeley.edu/benchmark/ */ class BigDataBenchmarkSuite extends HiveComparisonTest { - val testDataDirectory = new File("target/big-data-benchmark-testdata") + val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") val testTables = Seq( TestTable( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index ac87f2cb10d12..23ece7e7cf6e9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.hive.TestHiveContext +import org.apache.spark.sql.hive.test.TestHiveContext import org.scalatest.{BeforeAndAfterAll, FunSuite} class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { 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 6c91f40d0f925..edff38b901073 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,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeComman import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.Sort import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.test.TestHive /** * Allows the creations of tests that execute the same query against both hive @@ -78,7 +78,8 @@ abstract class HiveComparisonTest .map(name => new File(targetDir, s"$suiteName.$name")) /** The local directory with cached golden answer will be stored. */ - protected val answerCache = new File("src/test/resources/golden") + protected val answerCache = new File("src" + File.separator + "test" + + File.separator + "resources" + File.separator + "golden") if (!answerCache.exists) { answerCache.mkdir() } @@ -120,7 +121,7 @@ abstract class HiveComparisonTest protected val cacheDigest = java.security.MessageDigest.getInstance("MD5") protected def getMd5(str: String): String = { val digest = java.security.MessageDigest.getInstance("MD5") - digest.update(str.getBytes) + digest.update(str.getBytes("utf-8")) new java.math.BigInteger(1, digest.digest).toString(16) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index c3cfa3d25a5c2..0bb76f31c373d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.sql.hive.execution +import java.io.File + import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.test.TestHive /** * Runs the test cases that are included in the hive distribution. */ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // TODO: bundle in jar files... get from classpath - lazy val hiveQueryDir = TestHive.getHiveFile("ql/src/test/queries/clientpositive") + lazy val hiveQueryDir = TestHive.getHiveFile("ql" + File.separator + "src" + + File.separator + "test" + File.separator + "queries" + File.separator + "clientpositive") + def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) override def beforeAll() { 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 a09667ac84b01..d224d2ee600ec 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. 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 8883e5b16d4da..67594b57d3dfa 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 @@ -17,8 +17,8 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.TestHive -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ case class Data(a: Int, B: Int, n: Nested) case class Nested(a: Int, B: Int) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index d9ccb93e23923..25eca397462f7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.{HiveTableScan, TestHive} +import org.apache.spark.sql.hive.test.TestHive /* Implicit conversions */ import scala.collection.JavaConversions._ 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 843c681e0dc48..91ad59d7f82c0 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 @@ -22,11 +22,10 @@ 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.sql.hive.TestHive import org.apache.spark.util.Utils // Implicits -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { diff --git a/streaming/pom.xml b/streaming/pom.xml index 93b1c5a37aff9..6435224a14674 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -31,20 +31,6 @@ Spark Project Streaming http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index d3339063cc079..b4adf0e9651a8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import java.io.{ObjectInputStream, IOException, ObjectOutputStream} import org.apache.spark.Logging import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.streaming.dstream.{DStream, NetworkInputDStream, InputDStream} +import org.apache.spark.streaming.dstream.{DStream, ReceiverInputDStream, InputDStream} final private[streaming] class DStreamGraph extends Serializable with Logging { @@ -103,9 +103,9 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def getOutputStreams() = this.synchronized { outputStreams.toArray } - def getNetworkInputStreams() = this.synchronized { - inputStreams.filter(_.isInstanceOf[NetworkInputDStream[_]]) - .map(_.asInstanceOf[NetworkInputDStream[_]]) + def getReceiverInputStreams() = this.synchronized { + inputStreams.filter(_.isInstanceOf[ReceiverInputDStream[_]]) + .map(_.asInstanceOf[ReceiverInputDStream[_]]) .toArray } 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 e9a4f7ba22576..e0677b795cb94 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -31,12 +31,11 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat - import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receivers._ +import org.apache.spark.streaming.receiver.{ActorSupervisorStrategy, ActorReceiver, Receiver} import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.ui.StreamingTab import org.apache.spark.util.MetadataCleaner @@ -117,11 +116,6 @@ class StreamingContext private[streaming] ( } } - if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) { - throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; " - + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)") - } - private[streaming] val conf = sc.conf private[streaming] val env = SparkEnv.get @@ -139,7 +133,7 @@ class StreamingContext private[streaming] ( } } - private val nextNetworkInputStreamId = new AtomicInteger(0) + private val nextReceiverInputStreamId = new AtomicInteger(0) private[streaming] var checkpointDir: String = { if (isCheckpointPresent) { @@ -160,6 +154,10 @@ class StreamingContext private[streaming] ( private[streaming] val uiTab = new StreamingTab(this) + /** Register streaming source to metrics system */ + private val streamingSource = new StreamingSource(this) + SparkEnv.get.metricsSystem.registerSource(streamingSource) + /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { type CheckpointState = Value @@ -208,15 +206,26 @@ class StreamingContext private[streaming] ( if (isCheckpointPresent) cp_ else null } - private[streaming] def getNewNetworkStreamId() = nextNetworkInputStreamId.getAndIncrement() + private[streaming] def getNewReceiverStreamId() = nextReceiverInputStreamId.getAndIncrement() /** - * Create an input stream with any arbitrary user implemented network receiver. + * Create an input stream with any arbitrary user implemented receiver. * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * @param receiver Custom implementation of NetworkReceiver + * @param receiver Custom implementation of Receiver */ + @deprecated("Use receiverStream", "1.0.0") def networkStream[T: ClassTag]( - receiver: NetworkReceiver[T]): DStream[T] = { + receiver: Receiver[T]): ReceiverInputDStream[T] = { + receiverStream(receiver) + } + + /** + * Create an input stream with any arbitrary user implemented receiver. + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * @param receiver Custom implementation of Receiver + */ + def receiverStream[T: ClassTag]( + receiver: Receiver[T]): ReceiverInputDStream[T] = { new PluggableInputDStream[T](this, receiver) } @@ -236,9 +245,9 @@ class StreamingContext private[streaming] ( props: Props, name: String, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy - ): DStream[T] = { - networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) + supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy + ): ReceiverInputDStream[T] = { + receiverStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) } /** @@ -254,7 +263,7 @@ class StreamingContext private[streaming] ( hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[String] = { + ): ReceiverInputDStream[String] = { socketStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel) } @@ -273,7 +282,7 @@ class StreamingContext private[streaming] ( port: Int, converter: (InputStream) => Iterator[T], storageLevel: StorageLevel - ): DStream[T] = { + ): ReceiverInputDStream[T] = { new SocketInputDStream[T](this, hostname, port, converter, storageLevel) } @@ -292,7 +301,7 @@ class StreamingContext private[streaming] ( hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[T] = { + ): ReceiverInputDStream[T] = { new RawInputDStream[T](this, hostname, port, storageLevel) } @@ -310,7 +319,7 @@ class StreamingContext private[streaming] ( K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]: ClassTag - ] (directory: String): DStream[(K, V)] = { + ] (directory: String): InputDStream[(K, V)] = { new FileInputDStream[K, V, F](this, directory) } @@ -330,7 +339,7 @@ class StreamingContext private[streaming] ( K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]: ClassTag - ] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): DStream[(K, V)] = { + ] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): InputDStream[(K, V)] = { new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly) } @@ -356,7 +365,7 @@ class StreamingContext private[streaming] ( def queueStream[T: ClassTag]( queue: Queue[RDD[T]], oneAtATime: Boolean = true - ): DStream[T] = { + ): InputDStream[T] = { queueStream(queue, oneAtATime, sc.makeRDD(Seq[T](), 1)) } @@ -373,7 +382,7 @@ class StreamingContext private[streaming] ( queue: Queue[RDD[T]], oneAtATime: Boolean, defaultRDD: RDD[T] - ): DStream[T] = { + ): InputDStream[T] = { new QueueInputDStream(this, queue, oneAtATime, defaultRDD) } @@ -492,7 +501,8 @@ object StreamingContext extends Logging { private[streaming] val DEFAULT_CLEANER_TTL = 3600 - implicit def toPairDStreamFunctions[K: ClassTag, V: ClassTag](stream: DStream[(K,V)]) = { + implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { new PairDStreamFunctions[K, V](stream) } @@ -533,16 +543,10 @@ object StreamingContext extends Logging { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. */ - def jarOfClass(cls: Class[_]): Seq[String] = SparkContext.jarOfClass(cls) + def jarOfClass(cls: Class[_]): Option[String] = SparkContext.jarOfClass(cls) private[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { - // Set the default cleaner delay to an hour if not already set. - // This should be sufficient for even 1 second batch intervals. - if (MetadataCleaner.getDelaySeconds(conf) < 0) { - MetadataCleaner.setDelaySeconds(conf, DEFAULT_CLEANER_TTL) - } - val sc = new SparkContext(conf) - sc + new SparkContext(conf) } private[streaming] def createNewSparkContext( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala new file mode 100644 index 0000000000000..774adc3c23c21 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -0,0 +1,73 @@ +/* + * 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.streaming + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.metrics.source.Source +import org.apache.spark.streaming.ui.StreamingJobProgressListener + +private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { + val metricRegistry = new MetricRegistry + val sourceName = "%s.StreamingMetrics".format(ssc.sparkContext.appName) + + val streamingListener = ssc.uiTab.listener + + private def registerGauge[T](name: String, f: StreamingJobProgressListener => T, + defaultValue: T) { + metricRegistry.register(MetricRegistry.name("streaming", name), new Gauge[T] { + override def getValue: T = Option(f(streamingListener)).getOrElse(defaultValue) + }) + } + + // Gauge for number of network receivers + registerGauge("receivers", _.numReceivers, 0) + + // Gauge for number of total completed batches + registerGauge("totalCompletedBatches", _.numTotalCompletedBatches, 0L) + + // Gauge for number of unprocessed batches + registerGauge("unprocessedBatches", _.numUnprocessedBatches, 0L) + + // Gauge for number of waiting batches + registerGauge("waitingBatches", _.waitingBatches.size, 0L) + + // Gauge for number of running batches + registerGauge("runningBatches", _.runningBatches.size, 0L) + + // Gauge for number of retained completed batches + registerGauge("retainedCompletedBatches", _.retainedCompletedBatches.size, 0L) + + // 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) + + // 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) +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index 6a6b00a778b48..37b3b28fa01cb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -68,5 +68,5 @@ case class Time(private val millis: Long) { } object Time { - val ordering = Ordering.by((time: Time) => time.millis) + implicit val ordering = Ordering.by((time: Time) => time.millis) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 13e2bacc92edc..505e4431e4350 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -97,6 +97,10 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T] } object JavaDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.DStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaDStream]]. + */ implicit def fromDStream[T: ClassTag](dstream: DStream[T]): JavaDStream[T] = new JavaDStream[T](dstream) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala new file mode 100644 index 0000000000000..ae5e693df93d9 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala @@ -0,0 +1,41 @@ +/* + * 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.streaming.api.java + +import scala.language.implicitConversions +import scala.reflect.ClassTag + +import org.apache.spark.streaming.dstream.InputDStream + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.InputDStream]]. + */ +class JavaInputDStream[T](val inputDStream: InputDStream[T]) + (implicit override val classTag: ClassTag[T]) extends JavaDStream[T](inputDStream) { +} + +object JavaInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.InputDStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaInputDStream]]. + */ + implicit def fromInputDStream[T: ClassTag]( + inputDStream: InputDStream[T]): JavaInputDStream[T] = { + new JavaInputDStream[T](inputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala new file mode 100644 index 0000000000000..e6ff8a0cb545f --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala @@ -0,0 +1,43 @@ +/* + * 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.streaming.api.java + +import org.apache.spark.streaming.dstream.InputDStream + +import scala.language.implicitConversions +import scala.reflect.ClassTag + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.InputDStream]] of + * key-value pairs. + */ +class JavaPairInputDStream[K, V](val inputDStream: InputDStream[(K, V)])( + implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V] + ) extends JavaPairDStream[K, V](inputDStream) { +} + +object JavaPairInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.InputDStream]] of pairs to a + * Java-friendly [[org.apache.spark.streaming.api.java.JavaPairInputDStream]]. + */ + implicit def fromInputDStream[K: ClassTag, V: ClassTag]( + inputDStream: InputDStream[(K, V)]): JavaPairInputDStream[K, V] = { + new JavaPairInputDStream[K, V](inputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala new file mode 100644 index 0000000000000..294bc2be1898c --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala @@ -0,0 +1,43 @@ +/* + * 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.streaming.api.java + +import scala.language.implicitConversions +import scala.reflect.ClassTag + +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.ReceiverInputDStream]], the + * abstract class for defining any input stream that receives data over the network. + */ +class JavaPairReceiverInputDStream[K, V](val receiverInputDStream: ReceiverInputDStream[(K, V)]) + (implicit override val kClassTag: ClassTag[K], override implicit val vClassTag: ClassTag[V]) + extends JavaPairInputDStream[K, V](receiverInputDStream) { +} + +object JavaPairReceiverInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaReceiverInputDStream]]. + */ + implicit def fromReceiverInputDStream[K: ClassTag, V: ClassTag]( + receiverInputDStream: ReceiverInputDStream[(K, V)]): JavaPairReceiverInputDStream[K, V] = { + new JavaPairReceiverInputDStream[K, V](receiverInputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala new file mode 100644 index 0000000000000..8142dc59ea7c7 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.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.streaming.api.java + +import scala.language.implicitConversions +import scala.reflect.ClassTag + +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.ReceiverInputDStream]], the + * abstract class for defining any input stream that receives data over the network. + */ +class JavaReceiverInputDStream[T](val receiverInputDStream: ReceiverInputDStream[T]) + (implicit override val classTag: ClassTag[T]) extends JavaInputDStream[T](receiverInputDStream) { +} + +object JavaReceiverInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaReceiverInputDStream]]. + */ + implicit def fromReceiverInputDStream[T: ClassTag]( + receiverInputDStream: ReceiverInputDStream[T]): JavaReceiverInputDStream[T] = { + new JavaReceiverInputDStream[T](receiverInputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index c800602d0959b..75a3e9334e6d5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -35,7 +35,8 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.scheduler.StreamingListener import org.apache.hadoop.conf.Configuration -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.dstream.{PluggableInputDStream, ReceiverInputDStream, DStream} +import org.apache.spark.streaming.receiver.Receiver /** * A Java-friendly version of [[org.apache.spark.streaming.StreamingContext]] which is the main @@ -155,8 +156,10 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param port Port to connect to for receiving data * @param storageLevel Storage level to use for storing the received objects */ - def socketTextStream(hostname: String, port: Int, storageLevel: StorageLevel) - : JavaDStream[String] = { + def socketTextStream( + hostname: String, port: Int, + storageLevel: StorageLevel + ): JavaReceiverInputDStream[String] = { ssc.socketTextStream(hostname, port, storageLevel) } @@ -167,7 +170,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param hostname Hostname to connect to for receiving data * @param port Port to connect to for receiving data */ - def socketTextStream(hostname: String, port: Int): JavaDStream[String] = { + def socketTextStream(hostname: String, port: Int): JavaReceiverInputDStream[String] = { ssc.socketTextStream(hostname, port) } @@ -186,7 +189,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { port: Int, converter: JFunction[InputStream, java.lang.Iterable[T]], storageLevel: StorageLevel) - : JavaDStream[T] = { + : JavaReceiverInputDStream[T] = { def fn = (x: InputStream) => converter.call(x).toIterator implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] @@ -218,10 +221,11 @@ class JavaStreamingContext(val ssc: StreamingContext) { def rawSocketStream[T]( hostname: String, port: Int, - storageLevel: StorageLevel): JavaDStream[T] = { + storageLevel: StorageLevel): JavaReceiverInputDStream[T] = { implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel)) + JavaReceiverInputDStream.fromReceiverInputDStream( + ssc.rawSocketStream(hostname, port, storageLevel)) } /** @@ -233,10 +237,11 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param port Port to connect to for receiving data * @tparam T Type of the objects in the received blocks */ - def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = { + def rawSocketStream[T](hostname: String, port: Int): JavaReceiverInputDStream[T] = { implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port)) + JavaReceiverInputDStream.fromReceiverInputDStream( + ssc.rawSocketStream(hostname, port)) } /** @@ -249,7 +254,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @tparam V Value type for reading HDFS file * @tparam F Input format for reading HDFS file */ - def fileStream[K, V, F <: NewInputFormat[K, V]](directory: String): JavaPairDStream[K, V] = { + def fileStream[K, V, F <: NewInputFormat[K, V]]( + directory: String): JavaPairInputDStream[K, V] = { implicit val cmk: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val cmv: ClassTag[V] = @@ -275,7 +281,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { name: String, storageLevel: StorageLevel, supervisorStrategy: SupervisorStrategy - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name, storageLevel, supervisorStrategy) @@ -296,7 +302,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { props: Props, name: String, storageLevel: StorageLevel - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name, storageLevel) @@ -316,14 +322,14 @@ class JavaStreamingContext(val ssc: StreamingContext) { def actorStream[T]( props: Props, name: String - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name) } /** - * Creates an input stream from an queue of RDDs. In each batch, + * Create an input stream from an queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: changes to the queue after the stream is created will not be recognized. @@ -339,7 +345,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Creates an input stream from an queue of RDDs. In each batch, + * Create an input stream from an queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: changes to the queue after the stream is created will not be recognized. @@ -347,7 +353,10 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval * @tparam T Type of objects in the RDD */ - def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = { + def queueStream[T]( + queue: java.util.Queue[JavaRDD[T]], + oneAtATime: Boolean + ): JavaInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] @@ -356,7 +365,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Creates an input stream from an queue of RDDs. In each batch, + * Create an input stream from an queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: changes to the queue after the stream is created will not be recognized. @@ -368,7 +377,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def queueStream[T]( queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean, - defaultRDD: JavaRDD[T]): JavaDStream[T] = { + defaultRDD: JavaRDD[T]): JavaInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] @@ -376,6 +385,17 @@ class JavaStreamingContext(val ssc: StreamingContext) { ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd) } + /** + * Create an input stream with any arbitrary user implemented receiver. + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * @param receiver Custom implementation of Receiver + */ + def receiverStream[T](receiver: Receiver[T]): JavaReceiverInputDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + ssc.receiverStream(receiver) + } + /** * Create a unified DStream from multiple DStreams of the same type and same slide duration. */ 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 a7e5215437e54..4709a62381647 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 @@ -18,20 +18,19 @@ package org.apache.spark.streaming.dstream +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} + import scala.deprecated import scala.collection.mutable.HashMap import scala.reflect.ClassTag -import java.io.{IOException, ObjectInputStream, ObjectOutputStream} - -import org.apache.spark.Logging -import org.apache.spark.rdd.RDD +import org.apache.spark.{Logging, SparkException} +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.MetadataCleaner import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.streaming.Duration +import org.apache.spark.util.MetadataCleaner /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -144,7 +143,7 @@ abstract class DStream[T: ClassTag] ( */ private[streaming] def initialize(time: Time) { if (zeroTime != null && zeroTime != time) { - throw new Exception("ZeroTime is already initialized to " + zeroTime + throw new SparkException("ZeroTime is already initialized to " + zeroTime + ", cannot initialize it again to " + time) } zeroTime = time @@ -220,7 +219,7 @@ abstract class DStream[T: ClassTag] ( "which requires " + this.getClass.getSimpleName + " to remember generated RDDs for more " + "than " + rememberDuration.milliseconds / 1000 + " seconds. But Spark's metadata cleanup" + "delay is set to " + metadataCleanerDelay + " seconds, which is not sufficient. Please " + - "set the Java property 'spark.cleaner.delay' to more than " + + "set the Java cleaner delay to more than " + math.ceil(rememberDuration.milliseconds / 1000.0).toInt + " seconds." ) @@ -235,7 +234,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def setContext(s: StreamingContext) { if (ssc != null && ssc != s) { - throw new Exception("Context is already set in " + this + ", cannot set it again") + throw new SparkException("Context is already set in " + this + ", cannot set it again") } ssc = s logInfo("Set context for " + this) @@ -244,7 +243,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def setGraph(g: DStreamGraph) { if (graph != null && graph != g) { - throw new Exception("Graph is already set in " + this + ", cannot set it again") + throw new SparkException("Graph is already set in " + this + ", cannot set it again") } graph = g dependencies.foreach(_.setGraph(graph)) @@ -261,7 +260,7 @@ abstract class DStream[T: ClassTag] ( /** Checks whether the 'time' is valid wrt slideDuration for generating RDD */ private[streaming] def isTimeValid(time: Time): Boolean = { if (!isInitialized) { - throw new Exception (this + " has not been initialized") + throw new SparkException (this + " has not been initialized") } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) { logInfo("Time " + time + " is invalid as zeroTime is " + zeroTime + " and slideDuration is " + slideDuration + " and difference is " + (time - zeroTime)) @@ -340,13 +339,23 @@ abstract class DStream[T: ClassTag] ( * this to clear their own metadata along with the generated RDDs. */ private[streaming] def clearMetadata(time: Time) { + val unpersistData = ssc.conf.getBoolean("spark.streaming.unpersist", true) val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration)) logDebug("Clearing references to old RDDs: [" + oldRDDs.map(x => s"${x._1} -> ${x._2.id}").mkString(", ") + "]") generatedRDDs --= oldRDDs.keys - if (ssc.conf.getBoolean("spark.streaming.unpersist", false)) { + if (unpersistData) { logDebug("Unpersisting old RDDs: " + oldRDDs.values.map(_.id).mkString(", ")) - oldRDDs.values.foreach(_.unpersist(false)) + oldRDDs.values.foreach { rdd => + rdd.unpersist(false) + // Explicitly remove blocks of BlockRDD + rdd match { + case b: BlockRDD[_] => + logInfo("Removing blocks of RDD " + b + " of time " + time) + b.removeBlocks() + case _ => + } + } } logDebug("Cleared " + oldRDDs.size + " RDDs that were older than " + (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", ")) @@ -488,7 +497,8 @@ abstract class DStream[T: ClassTag] ( * the RDDs with `numPartitions` partitions (Spark's default number of partitions if * `numPartitions` not specified). */ - def countByValue(numPartitions: Int = ssc.sc.defaultParallelism): DStream[(T, Long)] = + def countByValue(numPartitions: Int = ssc.sc.defaultParallelism)(implicit ord: Ordering[T] = null) + : DStream[(T, Long)] = this.map(x => (x, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions) /** @@ -686,9 +696,10 @@ abstract class DStream[T: ClassTag] ( def countByValueAndWindow( windowDuration: Duration, slideDuration: Duration, - numPartitions: Int = ssc.sc.defaultParallelism - ): DStream[(T, Long)] = { - + numPartitions: Int = ssc.sc.defaultParallelism) + (implicit ord: Ordering[T] = null) + : DStream[(T, Long)] = + { this.map(x => (x, 1L)).reduceByKeyAndWindow( (x: Long, y: Long) => x + y, (x: Long, y: Long) => x - y, @@ -716,6 +727,9 @@ abstract class DStream[T: ClassTag] ( * Return all the RDDs between 'fromTime' to 'toTime' (both included) */ def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = { + if (!isInitialized) { + throw new SparkException(this + " has not been initialized") + } if (!(fromTime - zeroTime).isMultipleOf(slideDuration)) { logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 226844c2284e3..aa1993f0580a8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -30,7 +30,7 @@ import scala.reflect.ClassTag * FileInputDStream, a subclass of InputDStream, monitors a HDFS directory from the driver for * new files and generates RDDs with the new files. For implementing input streams * that requires running a receiver on the worker nodes, use - * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] as the parent class. + * [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] as the parent class. * * @param ssc_ Streaming context that will execute this input stream */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala deleted file mode 100644 index 5a249706b4d2f..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ /dev/null @@ -1,362 +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.streaming.dstream - -import java.nio.ByteBuffer -import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} - -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.concurrent.Await -import scala.reflect.ClassTag - -import akka.actor.{Actor, Props} -import akka.pattern.ask - -import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.rdd.{BlockRDD, RDD} -import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} -import org.apache.spark.streaming._ -import org.apache.spark.streaming.scheduler.{AddBlock, DeregisterReceiver, ReceivedBlockInfo, RegisterReceiver} -import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} -import org.apache.spark.util.{AkkaUtils, Utils} - -/** - * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] - * that has to start a receiver on worker nodes to receive external data. - * Specific implementations of NetworkInputDStream must - * define the getReceiver() function that gets the receiver object of type - * [[org.apache.spark.streaming.dstream.NetworkReceiver]] that will be sent - * to the workers to receive data. - * @param ssc_ Streaming context that will execute this input stream - * @tparam T Class type of the object of this stream - */ -abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) - extends InputDStream[T](ssc_) { - - /** Keeps all received blocks information */ - private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] - - /** This is an unique identifier for the network input stream. */ - val id = ssc.getNewNetworkStreamId() - - /** - * Gets the receiver object that will be sent to the worker nodes - * to receive data. This method needs to defined by any specific implementation - * of a NetworkInputDStream. - */ - def getReceiver(): NetworkReceiver[T] - - // Nothing to start or stop as both taken care of by the NetworkInputTracker. - def start() {} - - def stop() {} - - /** Ask NetworkInputTracker for received data blocks and generates RDDs with them. */ - override def compute(validTime: Time): Option[RDD[T]] = { - // If this is called for any time before the start time of the context, - // then this returns an empty RDD. This may happen when recovering from a - // master failure - if (validTime >= graph.startTime) { - val blockInfo = ssc.scheduler.networkInputTracker.getReceivedBlockInfo(id) - receivedBlockInfo(validTime) = blockInfo - val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) - Some(new BlockRDD[T](ssc.sc, blockIds)) - } else { - Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) - } - } - - /** Get information on received blocks. */ - private[streaming] def getReceivedBlockInfo(time: Time) = { - receivedBlockInfo(time) - } - - /** - * Clear metadata that are older than `rememberDuration` of this DStream. - * This is an internal method that should not be called directly. This - * implementation overrides the default implementation to clear received - * block information. - */ - private[streaming] override def clearMetadata(time: Time) { - super.clearMetadata(time) - val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) - receivedBlockInfo --= oldReceivedBlocks.keys - logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + - (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) - } -} - - -private[streaming] sealed trait NetworkReceiverMessage -private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage - -/** - * Abstract class of a receiver that can be run on worker nodes to receive external data. See - * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] for an explanation. - */ -abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging { - - /** Local SparkEnv */ - lazy protected val env = SparkEnv.get - - /** Remote Akka actor for the NetworkInputTracker */ - lazy protected val trackerActor = { - val ip = env.conf.get("spark.driver.host", "localhost") - val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) - env.actorSystem.actorSelection(url) - } - - /** Akka actor for receiving messages from the NetworkInputTracker in the driver */ - lazy protected val actor = env.actorSystem.actorOf( - Props(new NetworkReceiverActor()), "NetworkReceiver-" + streamId) - - /** Timeout for Akka actor messages */ - lazy protected val askTimeout = AkkaUtils.askTimeout(env.conf) - - /** Thread that starts the receiver and stays blocked while data is being received */ - lazy protected val receivingThread = Thread.currentThread() - - /** Exceptions that occurs while receiving data */ - protected lazy val exceptions = new ArrayBuffer[Exception] - - /** Identifier of the stream this receiver is associated with */ - protected var streamId: Int = -1 - - /** - * This method will be called to start receiving data. All your receiver - * starting code should be implemented by defining this function. - */ - protected def onStart() - - /** This method will be called to stop receiving data. */ - protected def onStop() - - /** Conveys a placement preference (hostname) for this receiver. */ - def getLocationPreference() : Option[String] = None - - /** - * Start the receiver. First is accesses all the lazy members to - * materialize them. Then it calls the user-defined onStart() method to start - * other threads, etc required to receiver the data. - */ - def start() { - try { - // Access the lazy vals to materialize them - env - actor - receivingThread - - // Call user-defined onStart() - logInfo("Starting receiver") - onStart() - - // Wait until interrupt is called on this thread - while(true) Thread.sleep(100000) - } catch { - case ie: InterruptedException => - logInfo("Receiving thread has been interrupted, receiver " + streamId + " stopped") - case e: Exception => - logError("Error receiving data in receiver " + streamId, e) - exceptions += e - } - - // Call user-defined onStop() - logInfo("Stopping receiver") - try { - onStop() - } catch { - case e: Exception => - logError("Error stopping receiver " + streamId, e) - exceptions += e - } - - val message = if (exceptions.isEmpty) { - null - } else if (exceptions.size == 1) { - val e = exceptions.head - "Exception in receiver " + streamId + ": " + e.getMessage + "\n" + e.getStackTraceString - } else { - "Multiple exceptions in receiver " + streamId + "(" + exceptions.size + "):\n" - exceptions.zipWithIndex.map { - case (e, i) => "Exception " + i + ": " + e.getMessage + "\n" + e.getStackTraceString - }.mkString("\n") - } - - logInfo("Deregistering receiver " + streamId) - val future = trackerActor.ask(DeregisterReceiver(streamId, message))(askTimeout) - Await.result(future, askTimeout) - logInfo("Deregistered receiver " + streamId) - env.actorSystem.stop(actor) - logInfo("Stopped receiver " + streamId) - } - - /** - * Stop the receiver. First it interrupts the main receiving thread, - * that is, the thread that called receiver.start(). - */ - def stop() { - // Stop receiving by interrupting the receiving thread - receivingThread.interrupt() - logInfo("Interrupted receiving thread " + receivingThread + " for stopping") - } - - /** - * Stop the receiver and reports exception to the tracker. - * This should be called whenever an exception is to be handled on any thread - * of the receiver. - */ - protected def stopOnError(e: Exception) { - logError("Error receiving data", e) - exceptions += e - stop() - } - - /** - * Push a block (as an ArrayBuffer filled with data) into the block manager. - */ - def pushBlock( - blockId: StreamBlockId, - arrayBuffer: ArrayBuffer[T], - metadata: Any, - level: StorageLevel - ) { - env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) - trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, arrayBuffer.size, metadata)) - logDebug("Pushed block " + blockId) - } - - /** - * Push a block (as bytes) into the block manager. - */ - def pushBlock( - blockId: StreamBlockId, - bytes: ByteBuffer, - metadata: Any, - level: StorageLevel - ) { - env.blockManager.putBytes(blockId, bytes, level) - trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, -1, metadata)) - } - - /** Set the ID of the DStream that this receiver is associated with */ - protected[streaming] def setStreamId(id: Int) { - streamId = id - } - - /** A helper actor that communicates with the NetworkInputTracker */ - private class NetworkReceiverActor extends Actor { - - override def preStart() { - val msg = RegisterReceiver( - streamId, NetworkReceiver.this.getClass.getSimpleName, Utils.localHostName(), self) - val future = trackerActor.ask(msg)(askTimeout) - Await.result(future, askTimeout) - logInfo("Registered receiver " + streamId) - } - - override def receive() = { - case StopReceiver => - logInfo("Received stop signal") - stop() - } - } - - /** - * Batches objects created by a [[org.apache.spark.streaming.dstream.NetworkReceiver]] and puts - * them into appropriately named blocks at regular intervals. This class starts two threads, - * one to periodically start a new batch and prepare the previous batch of as a block, - * the other to push the blocks into the block manager. - */ - class BlockGenerator(storageLevel: StorageLevel) - extends Serializable with Logging { - - case class Block(id: StreamBlockId, buffer: ArrayBuffer[T], metadata: Any = null) - - val clock = new SystemClock() - val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) - val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer, - "BlockGenerator") - val blockStorageLevel = storageLevel - val blocksForPushing = new ArrayBlockingQueue[Block](1000) - val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } - - var currentBuffer = new ArrayBuffer[T] - var stopped = false - - def start() { - blockIntervalTimer.start() - blockPushingThread.start() - logInfo("Started BlockGenerator") - } - - def stop() { - blockIntervalTimer.stop(false) - stopped = true - blockPushingThread.join() - logInfo("Stopped BlockGenerator") - } - - def += (obj: T): Unit = synchronized { - currentBuffer += obj - } - - private def updateCurrentBuffer(time: Long): Unit = synchronized { - try { - val newBlockBuffer = currentBuffer - currentBuffer = new ArrayBuffer[T] - if (newBlockBuffer.size > 0) { - val blockId = StreamBlockId(NetworkReceiver.this.streamId, time - blockInterval) - val newBlock = new Block(blockId, newBlockBuffer) - blocksForPushing.add(newBlock) - } - } catch { - case ie: InterruptedException => - logInfo("Block updating timer thread was interrupted") - case e: Exception => - NetworkReceiver.this.stopOnError(e) - } - } - - private def keepPushingBlocks() { - logInfo("Started block pushing thread") - try { - while(!stopped) { - Option(blocksForPushing.poll(100, TimeUnit.MILLISECONDS)) match { - case Some(block) => - NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) - case None => - } - } - // Push out the blocks that are still left - logInfo("Pushing out the last " + blocksForPushing.size() + " blocks") - while (!blocksForPushing.isEmpty) { - val block = blocksForPushing.take() - NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) - logInfo("Blocks left to push " + blocksForPushing.size()) - } - logInfo("Stopped blocks pushing thread") - } catch { - case ie: InterruptedException => - logInfo("Block pushing thread was interrupted") - case e: Exception => - NetworkReceiver.this.stopOnError(e) - } - } - } -} 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 354bc132dcdc0..826bf39e860e1 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 @@ -37,13 +37,13 @@ import org.apache.spark.streaming.{Time, Duration} * Import `org.apache.spark.streaming.StreamingContext._` at the top of your program to use * these functions. */ -class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) - extends Serializable { - +class PairDStreamFunctions[K, V](self: DStream[(K,V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K]) + extends Serializable +{ private[streaming] def ssc = self.ssc - private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) - = { + private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) = { new HashPartitioner(numPartitions) } @@ -576,7 +576,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, + saveAsHadoopFiles(prefix, suffix, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } @@ -607,7 +607,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, + saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } @@ -630,7 +630,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) self.foreachRDD(saveFunc) } - private def getKeyClass() = implicitly[ClassTag[K]].runtimeClass + private def keyClass: Class[_] = kt.runtimeClass - private def getValueClass() = implicitly[ClassTag[V]].runtimeClass + private def valueClass: Class[_] = vt.runtimeClass } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 6f9477020a459..186e1bf03a944 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -19,13 +19,14 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.StreamingContext import scala.reflect.ClassTag +import org.apache.spark.streaming.receiver.Receiver private[streaming] class PluggableInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, - receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) { + receiver: Receiver[T]) extends ReceiverInputDStream[T](ssc_) { - def getReceiver(): NetworkReceiver[T] = { + def getReceiver(): Receiver[T] = { receiver } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index dea0f26f908fb..e2925b9e03ec3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.StreamingContext @@ -28,6 +28,7 @@ import java.nio.ByteBuffer import java.nio.channels.{ReadableByteChannel, SocketChannel} import java.io.EOFException import java.util.concurrent.ArrayBlockingQueue +import org.apache.spark.streaming.receiver.Receiver /** @@ -42,21 +43,19 @@ class RawInputDStream[T: ClassTag]( host: String, port: Int, storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_ ) with Logging { + ) extends ReceiverInputDStream[T](ssc_ ) with Logging { - def getReceiver(): NetworkReceiver[T] = { - new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[NetworkReceiver[T]] + def getReceiver(): Receiver[T] = { + new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[Receiver[T]] } } private[streaming] class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel) - extends NetworkReceiver[Any] { + extends Receiver[Any](storageLevel) with Logging { var blockPushingThread: Thread = null - override def getLocationPreference = None - def onStart() { // Open a socket to the target address and keep reading from it logInfo("Connecting to " + host + ":" + port) @@ -73,9 +72,8 @@ class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel) var nextBlockNumber = 0 while (true) { val buffer = queue.take() - val blockId = StreamBlockId(streamId, nextBlockNumber) nextBlockNumber += 1 - pushBlock(blockId, buffer, null, storageLevel) + store(buffer) } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala new file mode 100644 index 0000000000000..75cabdbf8da26 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -0,0 +1,94 @@ +/* + * 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.streaming.dstream + +import scala.collection.mutable.HashMap +import scala.reflect.ClassTag + +import org.apache.spark.rdd.{BlockRDD, RDD} +import org.apache.spark.storage.BlockId +import org.apache.spark.streaming._ +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler.ReceivedBlockInfo + +/** + * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] + * that has to start a receiver on worker nodes to receive external data. + * Specific implementations of NetworkInputDStream must + * define `the getReceiver()` function that gets the receiver object of type + * [[org.apache.spark.streaming.receiver.Receiver]] that will be sent + * to the workers to receive data. + * @param ssc_ Streaming context that will execute this input stream + * @tparam T Class type of the object of this stream + */ +abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) + extends InputDStream[T](ssc_) { + + /** Keeps all received blocks information */ + private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] + + /** This is an unique identifier for the network input stream. */ + val id = ssc.getNewReceiverStreamId() + + /** + * Gets the receiver object that will be sent to the worker nodes + * to receive data. This method needs to defined by any specific implementation + * of a NetworkInputDStream. + */ + def getReceiver(): Receiver[T] + + // Nothing to start or stop as both taken care of by the ReceiverInputTracker. + def start() {} + + def stop() {} + + /** Ask ReceiverInputTracker for received data blocks and generates RDDs with them. */ + override def compute(validTime: Time): Option[RDD[T]] = { + // If this is called for any time before the start time of the context, + // then this returns an empty RDD. This may happen when recovering from a + // master failure + if (validTime >= graph.startTime) { + val blockInfo = ssc.scheduler.receiverTracker.getReceivedBlockInfo(id) + receivedBlockInfo(validTime) = blockInfo + val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) + Some(new BlockRDD[T](ssc.sc, blockIds)) + } else { + Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) + } + } + + /** Get information on received blocks. */ + private[streaming] def getReceivedBlockInfo(time: Time) = { + receivedBlockInfo(time) + } + + /** + * Clear metadata that are older than `rememberDuration` of this DStream. + * This is an internal method that should not be called directly. This + * implementation overrides the default implementation to clear received + * block information. + */ + private[streaming] override def clearMetadata(time: Time) { + super.clearMetadata(time) + val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) + receivedBlockInfo --= oldReceivedBlocks.keys + logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + + (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) + } +} + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 63d94d1cc670a..8b72bcf20653d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -24,7 +24,9 @@ import org.apache.spark.util.NextIterator import scala.reflect.ClassTag import java.io._ -import java.net.Socket +import java.net.{UnknownHostException, Socket} +import org.apache.spark.Logging +import org.apache.spark.streaming.receiver.Receiver private[streaming] class SocketInputDStream[T: ClassTag]( @@ -33,9 +35,9 @@ class SocketInputDStream[T: ClassTag]( port: Int, bytesToObjects: InputStream => Iterator[T], storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_) { + ) extends ReceiverInputDStream[T](ssc_) { - def getReceiver(): NetworkReceiver[T] = { + def getReceiver(): Receiver[T] = { new SocketReceiver(host, port, bytesToObjects, storageLevel) } } @@ -46,26 +48,45 @@ class SocketReceiver[T: ClassTag]( port: Int, bytesToObjects: InputStream => Iterator[T], storageLevel: StorageLevel - ) extends NetworkReceiver[T] { + ) extends Receiver[T](storageLevel) with Logging { - lazy protected val blockGenerator = new BlockGenerator(storageLevel) - - override def getLocationPreference = None + def onStart() { + // Start the thread that receives data over a connection + new Thread("Socket Receiver") { + setDaemon(true) + override def run() { receive() } + }.start() + } - protected def onStart() { - logInfo("Connecting to " + host + ":" + port) - val socket = new Socket(host, port) - logInfo("Connected to " + host + ":" + port) - blockGenerator.start() - val iterator = bytesToObjects(socket.getInputStream()) - while(iterator.hasNext) { - val obj = iterator.next - blockGenerator += obj - } + def onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false } - protected def onStop() { - blockGenerator.stop() + /** Create a socket connection and receive data until receiver is stopped */ + def receive() { + var socket: Socket = null + try { + logInfo("Connecting to " + host + ":" + port) + socket = new Socket(host, port) + logInfo("Connected to " + host + ":" + port) + val iterator = bytesToObjects(socket.getInputStream()) + while(!isStopped && iterator.hasNext) { + store(iterator.next) + } + logInfo("Stopped receiving") + restart("Retrying connecting to " + host + ":" + port) + } catch { + case e: java.net.ConnectException => + restart("Error connecting to " + host + ":" + port, e) + case t: Throwable => + restart("Error receiving data", t) + } finally { + if (socket != null) { + socket.close() + logInfo("Closed socket to " + host + ":" + port) + } + } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala similarity index 63% rename from streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala rename to streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index da0d364ae7bdb..743be58950c09 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -15,26 +15,27 @@ * limitations under the License. */ -package org.apache.spark.streaming.receivers +package org.apache.spark.streaming.receiver -import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } -import akka.actor.{ actorRef2Scala, ActorRef } -import akka.actor.{ PossiblyHarmful, OneForOneStrategy } -import akka.actor.SupervisorStrategy._ +import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.duration._ import scala.language.postfixOps import scala.reflect.ClassTag -import org.apache.spark.storage.{StorageLevel, StreamBlockId} -import org.apache.spark.streaming.dstream.NetworkReceiver - -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.mutable.ArrayBuffer +import akka.actor._ +import akka.actor.SupervisorStrategy.{Escalate, Restart} +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.storage.StorageLevel +import java.nio.ByteBuffer +import org.apache.spark.annotation.DeveloperApi -/** A helper with set of defaults for supervisor strategy */ -object ReceiverSupervisorStrategy { +/** + * :: DeveloperApi :: + * A helper with set of defaults for supervisor strategy + */ +@DeveloperApi +object ActorSupervisorStrategy { val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = 15 millis) { @@ -44,15 +45,16 @@ object ReceiverSupervisorStrategy { } /** + * :: DeveloperApi :: * A receiver trait to be mixed in with your Actor to gain access to * the API for pushing received data into Spark Streaming for being processed. * * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * * @example {{{ - * class MyActor extends Actor with Receiver{ + * class MyActor extends Actor with ActorHelper{ * def receive { - * case anything: String => pushBlock(anything) + * case anything: String => store(anything) * } * } * @@ -65,37 +67,54 @@ object ReceiverSupervisorStrategy { * to ensure the type safety, i.e parametrized type of push block and InputDStream * should be same. */ -trait Receiver { +@DeveloperApi +trait ActorHelper { self: Actor => // to ensure that this can be added to Actor classes only + /** Store an iterator of received data as a data block into Spark's memory. */ + def store[T](iter: Iterator[T]) { + println("Storing iterator") + context.parent ! IteratorData(iter) + } + /** - * Push an iterator received data into Spark Streaming for processing + * Store the bytes of received data as a data block into Spark's memory. Note + * that the data in the ByteBuffer must be serialized using the same serializer + * that Spark is configured to use. */ - def pushBlock[T: ClassTag](iter: Iterator[T]) { - context.parent ! Data(iter) + def store(bytes: ByteBuffer) { + context.parent ! ByteBufferData(bytes) } /** - * Push a single item of received data into Spark Streaming for processing + * Store a single item of received data to Spark's memory. + * These single items will be aggregated together into data blocks before + * being pushed into Spark's memory. */ - def pushBlock[T: ClassTag](data: T) { - context.parent ! Data(data) + def store[T](item: T) { + println("Storing item") + context.parent ! SingleItemData(item) } } /** + * :: DeveloperApi :: * Statistics for querying the supervisor about state of workers. Used in * conjunction with `StreamingContext.actorStream` and - * [[org.apache.spark.streaming.receivers.Receiver]]. + * [[org.apache.spark.streaming.receiver.ActorHelper]]. */ +@DeveloperApi case class Statistics(numberOfMsgs: Int, numberOfWorkers: Int, numberOfHiccups: Int, otherInfo: String) /** Case class to receive data sent by child actors */ -private[streaming] case class Data[T: ClassTag](data: T) +private[streaming] sealed trait ActorReceiverData +private[streaming] case class SingleItemData[T](item: T) extends ActorReceiverData +private[streaming] case class IteratorData[T](iterator: Iterator[T]) extends ActorReceiverData +private[streaming] case class ByteBufferData(bytes: ByteBuffer) extends ActorReceiverData /** * Provides Actors as receivers for receiving stream. @@ -117,16 +136,13 @@ private[streaming] case class Data[T: ClassTag](data: T) * }}} */ private[streaming] class ActorReceiver[T: ClassTag]( - props: Props, - name: String, - storageLevel: StorageLevel, - receiverSupervisorStrategy: SupervisorStrategy) - extends NetworkReceiver[T] { - - protected lazy val blocksGenerator: BlockGenerator = - new BlockGenerator(storageLevel) + props: Props, + name: String, + storageLevel: StorageLevel, + receiverSupervisorStrategy: SupervisorStrategy + ) extends Receiver[T](storageLevel) with Logging { - protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor), + protected lazy val supervisor = SparkEnv.get.actorSystem.actorOf(Props(new Supervisor), "Supervisor" + streamId) class Supervisor extends Actor { @@ -140,12 +156,18 @@ private[streaming] class ActorReceiver[T: ClassTag]( def receive = { - case Data(iter: Iterator[_]) => pushBlock(iter.asInstanceOf[Iterator[T]]) + case IteratorData(iterator) => + println("received iterator") + store(iterator.asInstanceOf[Iterator[T]]) - case Data(msg) => - blocksGenerator += msg.asInstanceOf[T] + case SingleItemData(msg) => + println("received single") + store(msg.asInstanceOf[T]) n.incrementAndGet + case ByteBufferData(bytes) => + store(bytes) + case props: Props => val worker = context.actorOf(props) logInfo("Started receiver worker at:" + worker.path) @@ -165,20 +187,13 @@ private[streaming] class ActorReceiver[T: ClassTag]( } } - protected def pushBlock(iter: Iterator[T]) { - val buffer = new ArrayBuffer[T] - buffer ++= iter - pushBlock(StreamBlockId(streamId, System.nanoTime()), buffer, null, storageLevel) - } - - protected def onStart() = { - blocksGenerator.start() + def onStart() = { supervisor logInfo("Supervision tree for receivers initialized at:" + supervisor.path) } - protected def onStop() = { + def onStop() = { supervisor ! PoisonPill } } 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 new file mode 100644 index 0000000000000..78cc2daa56e53 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -0,0 +1,142 @@ +/* + * 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.streaming.receiver + +import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} + +/** Listener object for BlockGenerator events */ +private[streaming] trait BlockGeneratorListener { + /** Called when a new block needs to be pushed */ + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) + /** Called when an error has occurred in BlockGenerator */ + def onError(message: String, throwable: Throwable) +} + +/** + * Generates batches of objects received by a + * [[org.apache.spark.streaming.receiver.Receiver]] and puts them into appropriately + * named blocks at regular intervals. This class starts two threads, + * one to periodically start a new batch and prepare the previous batch of as a block, + * the other to push the blocks into the block manager. + */ +private[streaming] class BlockGenerator( + listener: BlockGeneratorListener, + receiverId: Int, + conf: SparkConf + ) extends Logging { + + private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) + + private val clock = new SystemClock() + private val blockInterval = conf.getLong("spark.streaming.blockInterval", 200) + private val blockIntervalTimer = + new RecurringTimer(clock, blockInterval, updateCurrentBuffer, "BlockGenerator") + private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) + private val blocksForPushing = new ArrayBlockingQueue[Block](blockQueueSize) + private val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } + + @volatile private var currentBuffer = new ArrayBuffer[Any] + @volatile private var stopped = false + + /** Start block generating and pushing threads. */ + def start() { + blockIntervalTimer.start() + blockPushingThread.start() + logInfo("Started BlockGenerator") + } + + /** Stop all threads. */ + def stop() { + logInfo("Stopping BlockGenerator") + blockIntervalTimer.stop(interruptTimer = false) + stopped = true + logInfo("Waiting for block pushing thread") + blockPushingThread.join() + logInfo("Stopped BlockGenerator") + } + + /** + * Push a single data item into the buffer. All received data items + * will be periodically pushed into BlockManager. + */ + def += (data: Any): Unit = synchronized { + currentBuffer += data + } + + /** Change the buffer to which single records are added to. */ + private def updateCurrentBuffer(time: Long): Unit = synchronized { + try { + val newBlockBuffer = currentBuffer + currentBuffer = new ArrayBuffer[Any] + if (newBlockBuffer.size > 0) { + val blockId = StreamBlockId(receiverId, time - blockInterval) + val newBlock = new Block(blockId, newBlockBuffer) + blocksForPushing.put(newBlock) // put is blocking when queue is full + logDebug("Last element in " + blockId + " is " + newBlockBuffer.last) + } + } catch { + case ie: InterruptedException => + logInfo("Block updating timer thread was interrupted") + case t: Throwable => + reportError("Error in block updating thread", t) + } + } + + /** Keep pushing blocks to the BlockManager. */ + private def keepPushingBlocks() { + logInfo("Started block pushing thread") + try { + while(!stopped) { + Option(blocksForPushing.poll(100, TimeUnit.MILLISECONDS)) match { + case Some(block) => pushBlock(block) + case None => + } + } + // Push out the blocks that are still left + logInfo("Pushing out the last " + blocksForPushing.size() + " blocks") + while (!blocksForPushing.isEmpty) { + logDebug("Getting block ") + val block = blocksForPushing.take() + pushBlock(block) + logInfo("Blocks left to push " + blocksForPushing.size()) + } + logInfo("Stopped block pushing thread") + } catch { + case ie: InterruptedException => + logInfo("Block pushing thread was interrupted") + case t: Throwable => + reportError("Error in block pushing thread", t) + } + } + + private def reportError(message: String, t: Throwable) { + logError(message, t) + listener.onError(message, t) + } + + private def pushBlock(block: Block) { + listener.onPushBlock(block.id, block.buffer) + logInfo("Pushed block " + block.id) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala new file mode 100644 index 0000000000000..b310c22b3ab78 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -0,0 +1,277 @@ +/* + * 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.streaming.receiver + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Abstract class of a receiver that can be run on worker nodes to receive external data. A + * custom receiver can be defined by defining the functions onStart() and onStop(). onStart() + * should define the setup steps necessary to start receiving data, + * and onStop() should define the cleanup steps necessary to stop receiving data. + * + * A custom receiver in Scala would look like this. + * + * {{{ + * class MyReceiver(storageLevel: StorageLevel) extends NetworkReceiver[String](storageLevel) { + * def onStart() { + * // Setup stuff (start threads, open sockets, etc.) to start receiving data. + * // Must start new thread to receive data, as onStart() must be non-blocking. + * + * // Call store(...) in those threads to store received data into Spark's memory. + * + * // Call stop(...), restart(...) or reportError(...) on any thread based on how + * // different errors needs to be handled. + * + * // See corresponding method documentation for more details + * } + * + * def onStop() { + * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. + * } + * } + * }}} + * + * A custom receiver in Java would look like this. + * + * {{{ + * class MyReceiver extends Receiver { + * public MyReceiver(StorageLevel storageLevel) { + * super(storageLevel); + * } + * + * public void onStart() { + * // Setup stuff (start threads, open sockets, etc.) to start receiving data. + * // Must start new thread to receive data, as onStart() must be non-blocking. + * + * // Call store(...) in those threads to store received data into Spark's memory. + * + * // Call stop(...), restart(...) or reportError(...) on any thread based on how + * // different errors needs to be handled. + * + * // See corresponding method documentation for more details + * } + * + * public void onStop() { + * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. + * } + * } + * }}} + */ +@DeveloperApi +abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable { + + /** + * This method is called by the system when the receiver is started. This function + * must initialize all resources (threads, buffers, etc.) necessary for receiving data. + * This function must be non-blocking, so receiving the data must occur on a different + * thread. Received data can be stored with Spark by calling `store(data)`. + * + * If there are errors in threads started here, then following options can be done + * (i) `reportError(...)` can be called to report the error to the driver. + * The receiving of data will continue uninterrupted. + * (ii) `stop(...)` can be called to stop receiving data. This will call `onStop()` to + * clear up all resources allocated (threads, buffers, etc.) during `onStart()`. + * (iii) `restart(...)` can be called to restart the receiver. This will call `onStop()` + * immediately, and then `onStart()` after a delay. + */ + def onStart() + + /** + * This method is called by the system when the receiver is stopped. All resources + * (threads, buffers, etc.) setup in `onStart()` must be cleaned up in this method. + */ + def onStop() + + /** Override this to specify a preferred location (hostname). */ + def preferredLocation : Option[String] = None + + /** + * Store a single item of received data to Spark's memory. + * These single items will be aggregated together into data blocks before + * being pushed into Spark's memory. + */ + def store(dataItem: T) { + executor.pushSingle(dataItem) + } + + /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ + def store(dataBuffer: ArrayBuffer[T]) { + executor.pushArrayBuffer(dataBuffer, None, None) + } + + /** + * Store an ArrayBuffer of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(dataBuffer: ArrayBuffer[T], metadata: Any) { + executor.pushArrayBuffer(dataBuffer, Some(metadata), None) + } + + /** Store an iterator of received data as a data block into Spark's memory. */ + def store(dataIterator: Iterator[T]) { + executor.pushIterator(dataIterator, None, None) + } + + /** + * Store an iterator of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(dataIterator: java.util.Iterator[T], metadata: Any) { + executor.pushIterator(dataIterator, Some(metadata), None) + } + + /** Store an iterator of received data as a data block into Spark's memory. */ + def store(dataIterator: java.util.Iterator[T]) { + executor.pushIterator(dataIterator, None, None) + } + + /** + * Store an iterator of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(dataIterator: Iterator[T], metadata: Any) { + executor.pushIterator(dataIterator, Some(metadata), None) + } + + /** + * Store the bytes of received data as a data block into Spark's memory. Note + * that the data in the ByteBuffer must be serialized using the same serializer + * that Spark is configured to use. + */ + def store(bytes: ByteBuffer) { + executor.pushBytes(bytes, None, None) + } + + /** + * Store the bytes of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(bytes: ByteBuffer, metadata: Any) { + executor.pushBytes(bytes, Some(metadata), None) + } + + /** Report exceptions in receiving data. */ + def reportError(message: String, throwable: Throwable) { + executor.reportError(message, throwable) + } + + /** + * Restart the receiver. This method schedules the restart and returns + * immediately. The stopping and subsequent starting of the receiver + * (by calling `onStop()` and `onStart()`) is performed asynchronously + * in a background thread. The delay between the stopping and the starting + * is defined by the Spark configuration `spark.streaming.receiverRestartDelay`. + * The `message` will be reported to the driver. + */ + def restart(message: String) { + executor.restartReceiver(message) + } + + /** + * Restart the receiver. This method schedules the restart and returns + * immediately. The stopping and subsequent starting of the receiver + * (by calling `onStop()` and `onStart()`) is performed asynchronously + * in a background thread. The delay between the stopping and the starting + * is defined by the Spark configuration `spark.streaming.receiverRestartDelay`. + * The `message` and `exception` will be reported to the driver. + */ + def restart(message: String, error: Throwable) { + executor.restartReceiver(message, Some(error)) + } + + /** + * Restart the receiver. This method schedules the restart and returns + * immediately. The stopping and subsequent starting of the receiver + * (by calling `onStop()` and `onStart()`) is performed asynchronously + * in a background thread. + */ + def restart(message: String, error: Throwable, millisecond: Int) { + executor.restartReceiver(message, Some(error), millisecond) + } + + /** Stop the receiver completely. */ + def stop(message: String) { + executor.stop(message, None) + } + + /** Stop the receiver completely due to an exception */ + def stop(message: String, error: Throwable) { + executor.stop(message, Some(error)) + } + + /** Check if the receiver has started or not. */ + def isStarted(): Boolean = { + executor.isReceiverStarted() + } + + /** + * Check if receiver has been marked for stopping. Use this to identify when + * the receiving of data should be stopped. + */ + def isStopped(): Boolean = { + executor.isReceiverStopped() + } + + /** + * Get the unique identifier the receiver input stream that this + * receiver is associated with. + */ + def streamId = id + + /* + * ================= + * Private methods + * ================= + */ + + /** Identifier of the stream this receiver is associated with. */ + private var id: Int = -1 + + /** Handler object that runs the receiver. This is instantiated lazily in the worker. */ + private[streaming] var executor_ : ReceiverSupervisor = null + + /** Set the ID of the DStream that this receiver is associated with. */ + private[streaming] def setReceiverId(id_ : Int) { + id = id_ + } + + /** Attach Network Receiver executor to this receiver. */ + private[streaming] def attachExecutor(exec: ReceiverSupervisor) { + assert(executor_ == null) + executor_ = exec + } + + /** Get the attached executor. */ + private def executor = { + assert(executor_ != null, "Executor has not been attached to this receiver") + executor_ + } +} + 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 new file mode 100644 index 0000000000000..bf39d1e891cae --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala @@ -0,0 +1,23 @@ +/* + * 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.streaming.receiver + +/** Messages sent to the NetworkReceiver. */ +private[streaming] sealed trait ReceiverMessage +private[streaming] object StopReceiver extends ReceiverMessage + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala new file mode 100644 index 0000000000000..09be3a50d2dfa --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -0,0 +1,190 @@ +/* + * 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.streaming.receiver + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.storage.StreamBlockId +import java.util.concurrent.CountDownLatch +import scala.concurrent._ +import ExecutionContext.Implicits.global + +/** + * Abstract class that is responsible for supervising a Receiver in the worker. + * It provides all the necessary interfaces for handling the data received by the receiver. + */ +private[streaming] abstract class ReceiverSupervisor( + receiver: Receiver[_], + conf: SparkConf + ) extends Logging { + + /** Enumeration to identify current state of the StreamingContext */ + object ReceiverState extends Enumeration { + type CheckpointState = Value + val Initialized, Started, Stopped = Value + } + import ReceiverState._ + + // Attach the executor to the receiver + receiver.attachExecutor(this) + + /** Receiver id */ + protected val streamId = receiver.streamId + + /** Has the receiver been marked for stop. */ + private val stopLatch = new CountDownLatch(1) + + /** Time between a receiver is stopped and started again */ + private val defaultRestartDelay = conf.getInt("spark.streaming.receiverRestartDelay", 2000) + + /** Exception associated with the stopping of the receiver */ + @volatile protected var stoppingError: Throwable = null + + /** State of the receiver */ + @volatile private[streaming] var receiverState = Initialized + + /** Push a single data item to backend data store. */ + def pushSingle(data: Any) + + /** Store the bytes of received data as a data block into Spark's memory. */ + def pushBytes( + bytes: ByteBuffer, + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) + + /** Store a iterator of received data as a data block into Spark's memory. */ + def pushIterator( + iterator: Iterator[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) + + /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ + def pushArrayBuffer( + arrayBuffer: ArrayBuffer[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) + + /** Report errors. */ + def reportError(message: String, throwable: Throwable) + + /** Called when supervisor is started */ + protected def onStart() { } + + /** Called when supervisor is stopped */ + protected def onStop(message: String, error: Option[Throwable]) { } + + /** Called when receiver is started */ + protected def onReceiverStart() { } + + /** Called when receiver is stopped */ + protected def onReceiverStop(message: String, error: Option[Throwable]) { } + + /** Start the supervisor */ + def start() { + onStart() + startReceiver() + } + + /** Mark the supervisor and the receiver for stopping */ + def stop(message: String, error: Option[Throwable]) { + stoppingError = error.orNull + stopReceiver(message, error) + onStop(message, error) + stopLatch.countDown() + } + + /** Start receiver */ + def startReceiver(): Unit = synchronized { + try { + logInfo("Starting receiver") + receiver.onStart() + logInfo("Called receiver onStart") + onReceiverStart() + receiverState = Started + } catch { + case t: Throwable => + stop("Error starting receiver " + streamId, Some(t)) + } + } + + /** Stop receiver */ + def stopReceiver(message: String, error: Option[Throwable]): Unit = synchronized { + try { + logInfo("Stopping receiver with message: " + message + ": " + error.getOrElse("")) + receiverState = Stopped + receiver.onStop() + logInfo("Called receiver onStop") + onReceiverStop(message, error) + } catch { + case t: Throwable => + stop("Error stopping receiver " + streamId, Some(t)) + } + } + + /** Restart receiver with delay */ + def restartReceiver(message: String, error: Option[Throwable] = None) { + restartReceiver(message, error, defaultRestartDelay) + } + + /** Restart receiver with delay */ + def restartReceiver(message: String, error: Option[Throwable], delay: Int) { + Future { + logWarning("Restarting receiver with delay " + delay + " ms: " + message, + error.getOrElse(null)) + stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) + logDebug("Sleeping for " + delay) + Thread.sleep(delay) + logInfo("Starting receiver again") + startReceiver() + logInfo("Receiver started again") + } + } + + /** Check if receiver has been marked for stopping */ + def isReceiverStarted() = { + logDebug("state = " + receiverState) + receiverState == Started + } + + /** Check if receiver has been marked for stopping */ + def isReceiverStopped() = { + logDebug("state = " + receiverState) + receiverState == Stopped + } + + + /** Wait the thread until the supervisor is stopped */ + def awaitTermination() { + stopLatch.await() + logInfo("Waiting for executor stop is over") + if (stoppingError != null) { + logError("Stopped executor with error: " + stoppingError) + } else { + logWarning("Stopped executor without error") + } + if (stoppingError != null) { + throw stoppingError + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala new file mode 100644 index 0000000000000..ce8316bb14891 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -0,0 +1,186 @@ +/* + * 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.streaming.receiver + +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.concurrent.Await + +import akka.actor.{Actor, Props} +import akka.pattern.ask + +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.scheduler.DeregisterReceiver +import org.apache.spark.streaming.scheduler.AddBlock +import scala.Some +import org.apache.spark.streaming.scheduler.RegisterReceiver +import com.google.common.base.Throwables + +/** + * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] + * which provides all the necessary functionality for handling the data received by + * the receiver. Specifically, it creates a [[org.apache.spark.streaming.receiver.BlockGenerator]] + * object that is used to divide the received data stream into blocks of data. + */ +private[streaming] class ReceiverSupervisorImpl( + receiver: Receiver[_], + env: SparkEnv + ) extends ReceiverSupervisor(receiver, env.conf) with Logging { + + private val blockManager = env.blockManager + + private val storageLevel = receiver.storageLevel + + /** Remote Akka actor for the ReceiverTracker */ + private val trackerActor = { + val ip = env.conf.get("spark.driver.host", "localhost") + val port = env.conf.getInt("spark.driver.port", 7077) + val url = "akka.tcp://spark@%s:%s/user/ReceiverTracker".format(ip, port) + env.actorSystem.actorSelection(url) + } + + /** Timeout for Akka actor messages */ + private val askTimeout = AkkaUtils.askTimeout(env.conf) + + /** Akka actor for receiving messages from the ReceiverTracker in the driver */ + private val actor = env.actorSystem.actorOf( + Props(new Actor { + override def preStart() { + logInfo("Registered receiver " + streamId) + val msg = RegisterReceiver( + streamId, receiver.getClass.getSimpleName, Utils.localHostName(), self) + val future = trackerActor.ask(msg)(askTimeout) + Await.result(future, askTimeout) + } + + override def receive() = { + case StopReceiver => + logInfo("Received stop signal") + stop("Stopped by driver", None) + } + + def ref = self + }), "Receiver-" + streamId + "-" + System.currentTimeMillis()) + + /** Unique block ids if one wants to add blocks directly */ + private val newBlockId = new AtomicLong(System.currentTimeMillis()) + + /** Divides received data records into data blocks for pushing in BlockManager. */ + private val blockGenerator = new BlockGenerator(new BlockGeneratorListener { + def onError(message: String, throwable: Throwable) { + reportError(message, throwable) + } + + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) { + pushArrayBuffer(arrayBuffer, None, Some(blockId)) + } + }, streamId, env.conf) + + /** Push a single record of received data into block generator. */ + def pushSingle(data: Any) { + blockGenerator += (data) + } + + /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ + def pushArrayBuffer( + arrayBuffer: ArrayBuffer[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + val blockId = optionalBlockId.getOrElse(nextBlockId) + val time = System.currentTimeMillis + blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], + storageLevel, tellMaster = true) + logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") + reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) + } + + /** Store a iterator of received data as a data block into Spark's memory. */ + def pushIterator( + iterator: Iterator[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + val blockId = optionalBlockId.getOrElse(nextBlockId) + val time = System.currentTimeMillis + blockManager.put(blockId, iterator, storageLevel, tellMaster = true) + logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") + reportPushedBlock(blockId, -1, optionalMetadata) + } + + /** Store the bytes of received data as a data block into Spark's memory. */ + def pushBytes( + bytes: ByteBuffer, + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + val blockId = optionalBlockId.getOrElse(nextBlockId) + val time = System.currentTimeMillis + blockManager.putBytes(blockId, bytes, storageLevel, tellMaster = true) + logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") + reportPushedBlock(blockId, -1, optionalMetadata) + } + + /** Report pushed block */ + def reportPushedBlock(blockId: StreamBlockId, numRecords: Long, optionalMetadata: Option[Any]) { + val blockInfo = ReceivedBlockInfo(streamId, blockId, numRecords, optionalMetadata.orNull) + trackerActor ! AddBlock(blockInfo) + logDebug("Reported block " + blockId) + } + + /** Report error to the receiver tracker */ + def reportError(message: String, error: Throwable) { + val errorString = Option(error).map(Throwables.getStackTraceAsString).getOrElse("") + trackerActor ! ReportError(streamId, message, errorString) + logWarning("Reported error " + message + " - " + error) + } + + override protected def onStart() { + blockGenerator.start() + } + + override protected def onStop(message: String, error: Option[Throwable]) { + blockGenerator.stop() + env.actorSystem.stop(actor) + } + + override protected def onReceiverStart() { + val msg = RegisterReceiver( + streamId, receiver.getClass.getSimpleName, Utils.localHostName(), actor) + val future = trackerActor.ask(msg)(askTimeout) + Await.result(future, askTimeout) + } + + override protected def onReceiverStop(message: String, error: Option[Throwable]) { + logInfo("Deregistering receiver " + streamId) + val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") + val future = trackerActor.ask( + DeregisterReceiver(streamId, message, errorString))(askTimeout) + Await.result(future, askTimeout) + logInfo("Stopped receiver " + streamId) + } + + /** Generate new block ID */ + private def nextBlockId = StreamBlockId(streamId, newBlockId.getAndIncrement) +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 9c69a2a4e21f5..a68aecb881117 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -18,8 +18,10 @@ package org.apache.spark.streaming.scheduler import org.apache.spark.streaming.Time +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * Class having information on completed batches. * @param batchTime Time of the batch * @param submissionTime Clock time of when jobs of this batch was submitted to @@ -27,6 +29,7 @@ import org.apache.spark.streaming.Time * @param processingStartTime Clock time of when the first job of this batch started processing * @param processingEndTime Clock time of when the last job of this batch finished processing */ +@DeveloperApi case class BatchInfo( batchTime: Time, receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index e564eccba2df5..374848358e700 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -38,6 +38,7 @@ private[streaming] class JobGenerator(jobScheduler: JobScheduler) extends Logging { private val ssc = jobScheduler.ssc + private val conf = ssc.conf private val graph = ssc.graph val clock = { @@ -93,26 +94,31 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = 10 * ssc.graph.batchDuration.milliseconds + val stopTimeout = conf.getLong( + "spark.streaming.gracefulStopTimeout", + 10 * ssc.graph.batchDuration.milliseconds + ) val pollTime = 100 // To prevent graceful stop to get stuck permanently def hasTimedOut = { val timedOut = System.currentTimeMillis() - timeWhenStopStarted > stopTimeout - if (timedOut) logWarning("Timed out while stopping the job generator") + if (timedOut) { + logWarning("Timed out while stopping the job generator (timeout = " + stopTimeout + ")") + } timedOut } // Wait until all the received blocks in the network input tracker has // been consumed by network input DStreams, and jobs have been generated with them logInfo("Waiting for all received blocks to be consumed for job generation") - while(!hasTimedOut && jobScheduler.networkInputTracker.hasMoreReceivedBlockIds) { + while(!hasTimedOut && jobScheduler.receiverTracker.hasMoreReceivedBlockIds) { Thread.sleep(pollTime) } logInfo("Waited for all received blocks to be consumed for job generation") // Stop generating jobs - val stopTime = timer.stop(false) + val stopTime = timer.stop(interruptTimer = false) graph.stop() logInfo("Stopped generation timer") @@ -214,7 +220,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { case Success(jobs) => - val receivedBlockInfo = graph.getNetworkInputStreams.map { stream => + val receivedBlockInfo = graph.getReceiverInputStreams.map { stream => val streamId = stream.id val receivedBlockInfo = stream.getReceivedBlockInfo(time) (streamId, receivedBlockInfo) 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 d9ada99b472ac..1b034b9fb187c 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 @@ -46,7 +46,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { // These two are created only when scheduler starts. // eventActor not being null means the scheduler has been started and not stopped - var networkInputTracker: NetworkInputTracker = null + var receiverTracker: ReceiverTracker = null private var eventActor: ActorRef = null @@ -61,8 +61,8 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { }), "JobScheduler") listenerBus.start() - networkInputTracker = new NetworkInputTracker(ssc) - networkInputTracker.start() + receiverTracker = new ReceiverTracker(ssc) + receiverTracker.start() jobGenerator.start() logInfo("Started JobScheduler") } @@ -72,7 +72,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { logDebug("Stopping JobScheduler") // First, stop receiving - networkInputTracker.stop() + receiverTracker.stop() // Second, stop generating jobs. If it has to process all received data, // then this will wait for all the processing through JobScheduler to be over. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala new file mode 100644 index 0000000000000..d7e39c528c519 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala @@ -0,0 +1,37 @@ +/* + * 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.streaming.scheduler + +import akka.actor.ActorRef +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Class having information about a receiver + */ +@DeveloperApi +case class ReceiverInfo( + streamId: Int, + name: String, + private[streaming] val actor: ActorRef, + active: Boolean, + location: String, + lastErrorMessage: String = "", + lastError: String = "" + ) { +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala similarity index 60% rename from streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala rename to streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 438e72a7ced89..5307fe189d717 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -21,21 +21,15 @@ import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} import scala.language.existentials import akka.actor._ - import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.SparkContext._ import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.streaming.dstream.{NetworkReceiver, StopReceiver} +import org.apache.spark.streaming.receiver.{Receiver, ReceiverSupervisorImpl, StopReceiver} import org.apache.spark.util.AkkaUtils -/** Information about receiver */ -case class ReceiverInfo(streamId: Int, typ: String, location: String) { - override def toString = s"$typ-$streamId" -} - -/** Information about blocks received by the network receiver */ -case class ReceivedBlockInfo( +/** Information about blocks received by the receiver */ +private[streaming] case class ReceivedBlockInfo( streamId: Int, blockId: StreamBlockId, numRecords: Long, @@ -43,20 +37,21 @@ case class ReceivedBlockInfo( ) /** - * Messages used by the NetworkReceiver and the NetworkInputTracker to communicate + * Messages used by the NetworkReceiver and the ReceiverTracker to communicate * with each other. */ -private[streaming] sealed trait NetworkInputTrackerMessage +private[streaming] sealed trait ReceiverTrackerMessage private[streaming] case class RegisterReceiver( streamId: Int, typ: String, host: String, receiverActor: ActorRef - ) extends NetworkInputTrackerMessage + ) extends ReceiverTrackerMessage private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) - extends NetworkInputTrackerMessage -private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) - extends NetworkInputTrackerMessage + extends ReceiverTrackerMessage +private[streaming] case class ReportError(streamId: Int, message: String, error: String) +private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, error: String) + extends ReceiverTrackerMessage /** * This class manages the execution of the receivers of NetworkInputDStreams. Instance of @@ -64,12 +59,12 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) * has been called because it needs the final set of input streams at the time of instantiation. */ private[streaming] -class NetworkInputTracker(ssc: StreamingContext) extends Logging { +class ReceiverTracker(ssc: StreamingContext) extends Logging { - val networkInputStreams = ssc.graph.getNetworkInputStreams() - val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) - val receiverExecutor = new ReceiverExecutor() - val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] + val receiverInputStreams = ssc.graph.getReceiverInputStreams() + val receiverInputStreamMap = Map(receiverInputStreams.map(x => (x.id, x)): _*) + val receiverExecutor = new ReceiverLauncher() + val receiverInfo = new HashMap[Int, ReceiverInfo] with SynchronizedMap[Int, ReceiverInfo] val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] val timeout = AkkaUtils.askTimeout(ssc.conf) @@ -83,27 +78,27 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { /** Start the actor and receiver execution thread. */ def start() = synchronized { if (actor != null) { - throw new SparkException("NetworkInputTracker already started") + throw new SparkException("ReceiverTracker already started") } - if (!networkInputStreams.isEmpty) { - actor = ssc.env.actorSystem.actorOf(Props(new NetworkInputTrackerActor), - "NetworkInputTracker") + if (!receiverInputStreams.isEmpty) { + actor = ssc.env.actorSystem.actorOf(Props(new ReceiverTrackerActor), + "ReceiverTracker") receiverExecutor.start() - logInfo("NetworkInputTracker started") + logInfo("ReceiverTracker started") } } /** Stop the receiver execution thread. */ def stop() = synchronized { - if (!networkInputStreams.isEmpty && actor != null) { + if (!receiverInputStreams.isEmpty && actor != null) { // First, stop the receivers receiverExecutor.stop() // Finally, stop the actor ssc.env.actorSystem.stop(actor) actor = null - logInfo("NetworkInputTracker stopped") + logInfo("ReceiverTracker stopped") } } @@ -126,20 +121,32 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { receiverActor: ActorRef, sender: ActorRef ) { - if (!networkInputStreamMap.contains(streamId)) { + if (!receiverInputStreamMap.contains(streamId)) { throw new Exception("Register received for unexpected id " + streamId) } - receiverInfo += ((streamId, receiverActor)) - ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted( - ReceiverInfo(streamId, typ, host) - )) - logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) + receiverInfo(streamId) = ReceiverInfo( + streamId, s"${typ}-${streamId}", receiverActor, true, host) + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) + logInfo("Registered receiver for stream " + streamId + " from " + sender.path.address) } /** Deregister a receiver */ - def deregisterReceiver(streamId: Int, message: String) { - receiverInfo -= streamId - logError("Deregistered receiver for network stream " + streamId + " with message:\n" + message) + def deregisterReceiver(streamId: Int, message: String, error: String) { + val newReceiverInfo = receiverInfo.get(streamId) match { + case Some(oldInfo) => + oldInfo.copy(actor = null, active = false, lastErrorMessage = message, lastError = error) + case None => + logWarning("No prior receiver info") + ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) + } + receiverInfo(streamId) = newReceiverInfo + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(receiverInfo(streamId))) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logError(s"Deregistered receiver for stream $streamId: $messageWithError") } /** Add new blocks for the given stream */ @@ -149,27 +156,48 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { receivedBlockInfo.blockId) } + /** Report error sent by a receiver */ + def reportError(streamId: Int, message: String, error: String) { + val newReceiverInfo = receiverInfo.get(streamId) match { + case Some(oldInfo) => + oldInfo.copy(lastErrorMessage = message, lastError = error) + case None => + logWarning("No prior receiver info") + ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) + } + receiverInfo(streamId) = newReceiverInfo + ssc.scheduler.listenerBus.post(StreamingListenerReceiverError(receiverInfo(streamId))) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logWarning(s"Error reported by receiver for stream $streamId: $messageWithError") + } + /** Check if any blocks are left to be processed */ def hasMoreReceivedBlockIds: Boolean = { !receivedBlockInfo.values.forall(_.isEmpty) } /** Actor to receive messages from the receivers. */ - private class NetworkInputTrackerActor extends Actor { + private class ReceiverTrackerActor extends Actor { def receive = { case RegisterReceiver(streamId, typ, host, receiverActor) => registerReceiver(streamId, typ, host, receiverActor, sender) sender ! true case AddBlock(receivedBlockInfo) => addBlocks(receivedBlockInfo) - case DeregisterReceiver(streamId, message) => - deregisterReceiver(streamId, message) + case ReportError(streamId, message, error) => + reportError(streamId, message, error) + case DeregisterReceiver(streamId, message, error) => + deregisterReceiver(streamId, message, error) sender ! true } } /** This thread class runs all the receivers on the cluster. */ - class ReceiverExecutor { + class ReceiverLauncher { @transient val env = ssc.env @transient val thread = new Thread() { override def run() { @@ -177,7 +205,7 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { SparkEnv.set(env) startReceivers() } catch { - case ie: InterruptedException => logInfo("ReceiverExecutor interrupted") + case ie: InterruptedException => logInfo("ReceiverLauncher interrupted") } } } @@ -203,37 +231,38 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } /** - * Get the receivers from the NetworkInputDStreams, distributes them to the + * Get the receivers from the ReceiverInputDStreams, distributes them to the * worker nodes as a parallel collection, and runs them. */ private def startReceivers() { - val receivers = networkInputStreams.map(nis => { + val receivers = receiverInputStreams.map(nis => { val rcvr = nis.getReceiver() - rcvr.setStreamId(nis.id) + rcvr.setReceiverId(nis.id) rcvr }) // Right now, we only honor preferences if all receivers have them - val hasLocationPreferences = receivers.map(_.getLocationPreference().isDefined) - .reduce(_ && _) + val hasLocationPreferences = receivers.map(_.preferredLocation.isDefined).reduce(_ && _) // Create the parallel collection of receivers to distributed them on the worker nodes val tempRDD = if (hasLocationPreferences) { - val receiversWithPreferences = - receivers.map(r => (r, Seq(r.getLocationPreference().toString))) - ssc.sc.makeRDD[NetworkReceiver[_]](receiversWithPreferences) - } - else { + val receiversWithPreferences = receivers.map(r => (r, Seq(r.preferredLocation.get))) + ssc.sc.makeRDD[Receiver[_]](receiversWithPreferences) + } else { ssc.sc.makeRDD(receivers, receivers.size) } // Function to start the receiver on the worker node - val startReceiver = (iterator: Iterator[NetworkReceiver[_]]) => { + val startReceiver = (iterator: Iterator[Receiver[_]]) => { if (!iterator.hasNext) { - throw new Exception("Could not start receiver as details not found.") + throw new SparkException( + "Could not start receiver as object not found.") } - iterator.next().start() + val receiver = iterator.next() + val executor = new ReceiverSupervisorImpl(receiver, SparkEnv.get) + executor.start() + executor.awaitTermination() } // Run the dummy Spark job to ensure that all slaves have registered. // This avoids all the receivers to be scheduled on the same node. @@ -250,7 +279,8 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { /** Stops the receivers. */ private def stopReceivers() { // Signal the receivers to stop - receiverInfo.values.foreach(_ ! StopReceiver) + receiverInfo.values.flatMap { info => Option(info.actor)} + .foreach { _ ! StopReceiver } logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 5db40ebbeb1de..ed1aa114e19d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -18,44 +18,75 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.Queue + import org.apache.spark.util.Distribution +import org.apache.spark.annotation.DeveloperApi -/** Base trait for events related to StreamingListener */ +/** + * :: DeveloperApi :: + * Base trait for events related to StreamingListener + */ +@DeveloperApi sealed trait StreamingListenerEvent +@DeveloperApi case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent + +@DeveloperApi case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent + +@DeveloperApi case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent + +@DeveloperApi case class StreamingListenerReceiverStarted(receiverInfo: ReceiverInfo) extends StreamingListenerEvent +@DeveloperApi +case class StreamingListenerReceiverError(receiverInfo: ReceiverInfo) + extends StreamingListenerEvent + +@DeveloperApi +case class StreamingListenerReceiverStopped(receiverInfo: ReceiverInfo) + extends StreamingListenerEvent + /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object StreamingListenerShutdown extends StreamingListenerEvent /** + * :: DeveloperApi :: * A listener interface for receiving information about an ongoing streaming * computation. */ +@DeveloperApi trait StreamingListener { /** Called when a receiver has been started */ def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { } + /** Called when a receiver has reported an error */ + def onReceiverError(receiverError: StreamingListenerReceiverError) { } + + /** Called when a receiver has been stopped */ + def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { } + /** Called when a batch of jobs has been submitted for processing. */ def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { } - /** Called when processing of a batch of jobs has completed. */ - def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } - /** Called when processing of a batch of jobs has started. */ def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { } + + /** Called when processing of a batch of jobs has completed. */ + def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } } /** + * :: DeveloperApi :: * A simple StreamingListener that logs summary statistics across Spark Streaming batches * @param numBatchInfos Number of last batches to consider for generating statistics (default: 10) */ +@DeveloperApi class StatsReportListener(numBatchInfos: Int = 10) extends StreamingListener { // Queue containing latest completed batches val batchInfos = new Queue[BatchInfo]() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala index ea03dfc7bfeea..398724d9e8130 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala @@ -40,6 +40,10 @@ private[spark] class StreamingListenerBus() extends Logging { event match { case receiverStarted: StreamingListenerReceiverStarted => listeners.foreach(_.onReceiverStarted(receiverStarted)) + case receiverError: StreamingListenerReceiverError => + listeners.foreach(_.onReceiverError(receiverError)) + case receiverStopped: StreamingListenerReceiverStopped => + listeners.foreach(_.onReceiverStopped(receiverStopped)) case batchSubmitted: StreamingListenerBatchSubmitted => listeners.foreach(_.onBatchSubmitted(batchSubmitted)) case batchStarted: StreamingListenerBatchStarted => 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 8b025b09ed34d..f61069b56db5e 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 @@ -23,12 +23,13 @@ import scala.collection.mutable.{Queue, HashMap} import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted import org.apache.spark.streaming.scheduler.BatchInfo -import org.apache.spark.streaming.scheduler.ReceiverInfo import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted import org.apache.spark.util.Distribution +import org.apache.spark.Logging -private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends StreamingListener { +private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) + extends StreamingListener { private val waitingBatchInfos = new HashMap[Time, BatchInfo] private val runningBatchInfos = new HashMap[Time, BatchInfo] @@ -39,9 +40,21 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St val batchDuration = ssc.graph.batchDuration.milliseconds - override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) = { + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { synchronized { - receiverInfos.put(receiverStarted.receiverInfo.streamId, receiverStarted.receiverInfo) + receiverInfos(receiverStarted.receiverInfo.streamId) = receiverStarted.receiverInfo + } + } + + override def onReceiverError(receiverError: StreamingListenerReceiverError) { + synchronized { + receiverInfos(receiverError.receiverInfo.streamId) = receiverError.receiverInfo + } + } + + override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { + synchronized { + receiverInfos(receiverStopped.receiverInfo.streamId) = receiverStopped.receiverInfo } } @@ -62,8 +75,8 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St totalCompletedBatches += 1L } - def numNetworkReceivers = synchronized { - ssc.graph.getNetworkInputStreams().size + def numReceivers = synchronized { + ssc.graph.getReceiverInputStreams().size } def numTotalCompletedBatches: Long = synchronized { @@ -101,7 +114,7 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St def receivedRecordsDistributions: Map[Int, Option[Distribution]] = synchronized { val latestBatchInfos = retainedBatches.reverse.take(batchInfoLimit) val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) - (0 until numNetworkReceivers).map { receiverId => + (0 until numReceivers).map { receiverId => val blockInfoOfParticularReceiver = latestBlockInfos.map { batchInfo => batchInfo.get(receiverId).getOrElse(Array.empty) } @@ -117,11 +130,11 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St def lastReceivedBatchRecords: Map[Int, Long] = { val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo) lastReceivedBlockInfoOption.map { lastReceivedBlockInfo => - (0 until numNetworkReceivers).map { receiverId => + (0 until numReceivers).map { receiverId => (receiverId, lastReceivedBlockInfo(receiverId).map(_.numRecords).sum) }.toMap }.getOrElse { - (0 until numNetworkReceivers).map(receiverId => (receiverId, 0L)).toMap + (0 until numReceivers).map(receiverId => (receiverId, 0L)).toMap } } 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 6607437db560a..451b23e01c995 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 @@ -40,7 +40,7 @@ private[ui] class StreamingPage(parent: StreamingTab) val content = generateBasicStats() ++

++

Statistics over last {listener.retainedCompletedBatches.size} processed batches

++ - generateNetworkStatsTable() ++ + generateReceiverStats() ++ generateBatchStatsTable() UIUtils.headerSparkPage( content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000)) @@ -57,7 +57,7 @@ private[ui] class StreamingPage(parent: StreamingTab) Time since start: {formatDurationVerbose(timeSinceStart)}
  • - Network receivers: {listener.numNetworkReceivers} + Network receivers: {listener.numReceivers}
  • Batch interval: {formatDurationVerbose(listener.batchDuration)} @@ -71,32 +71,40 @@ private[ui] class StreamingPage(parent: StreamingTab) } - /** Generate stats of data received over the network the streaming program */ - private def generateNetworkStatsTable(): Seq[Node] = { + /** Generate stats of data received by the receivers in the streaming program */ + private def generateReceiverStats(): Seq[Node] = { val receivedRecordDistributions = listener.receivedRecordsDistributions val lastBatchReceivedRecord = listener.lastReceivedBatchRecords val table = if (receivedRecordDistributions.size > 0) { val headerRow = Seq( "Receiver", + "Status", "Location", "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", "Minimum rate\n[records/sec]", - "25th percentile rate\n[records/sec]", "Median rate\n[records/sec]", - "75th percentile rate\n[records/sec]", - "Maximum rate\n[records/sec]" + "Maximum rate\n[records/sec]", + "Last Error" ) - val dataRows = (0 until listener.numNetworkReceivers).map { receiverId => + val dataRows = (0 until listener.numReceivers).map { receiverId => val receiverInfo = listener.receiverInfo(receiverId) - val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") + val receiverName = receiverInfo.map(_.name).getOrElse(s"Receiver-$receiverId") + val receiverActive = receiverInfo.map { info => + if (info.active) "ACTIVE" else "INACTIVE" + }.getOrElse(emptyCell) val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) - val receiverLastBatchRecords = formatDurationVerbose(lastBatchReceivedRecord(receiverId)) + val receiverLastBatchRecords = formatNumber(lastBatchReceivedRecord(receiverId)) val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => - d.getQuantiles().map(r => formatDurationVerbose(r.toLong)) + d.getQuantiles(Seq(0.0, 0.5, 1.0)).map(r => formatNumber(r.toLong)) }.getOrElse { Seq(emptyCell, emptyCell, emptyCell, emptyCell, emptyCell) } - Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats + val receiverLastError = listener.receiverInfo(receiverId).map { info => + val msg = s"${info.lastErrorMessage} - ${info.lastError}" + if (msg.size > 100) msg.take(97) + "..." else msg + }.getOrElse(emptyCell) + Seq(receiverName, receiverActive, receiverLocation, receiverLastBatchRecords) ++ + receivedRecordStats ++ Seq(receiverLastError) } Some(listingTable(headerRow, dataRows)) } else { @@ -104,8 +112,8 @@ private[ui] class StreamingPage(parent: StreamingTab) } val content = -
    Network Input Statistics
    ++ -
    {table.getOrElse("No network receivers")}
    +
    Receiver Statistics
    ++ +
    {table.getOrElse("No receivers")}
    content } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index e016377c94c0d..1a616a0434f2c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -77,7 +77,9 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: def stop(interruptTimer: Boolean): Long = synchronized { if (!stopped) { stopped = true - if (interruptTimer) thread.interrupt() + if (interruptTimer) { + thread.interrupt() + } thread.join() logInfo("Stopped timer for " + name + " after time " + prevTime) } 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 a0b1bbc34fa7c..f9bfb9b7444cc 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.streaming.api.java.*; import scala.Tuple2; import org.junit.Assert; @@ -36,10 +37,6 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaDStreamLike; -import org.apache.spark.streaming.api.java.JavaPairDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -1668,7 +1665,7 @@ public Integer call(String s) throws Exception { // InputStream functionality is deferred to the existing Scala tests. @Test public void testSocketTextStream() { - JavaDStream test = ssc.socketTextStream("localhost", 12345); + JavaReceiverInputDStream test = ssc.socketTextStream("localhost", 12345); } @Test @@ -1701,6 +1698,6 @@ public void testTextFileStream() { @Test public void testRawSocketStream() { - JavaDStream test = ssc.rawSocketStream("localhost", 12345); + JavaReceiverInputDStream test = ssc.rawSocketStream("localhost", 12345); } } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java new file mode 100644 index 0000000000000..1b0787fe69dec --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java @@ -0,0 +1,144 @@ +/* + * 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.streaming; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import static org.junit.Assert.*; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.receiver.Receiver; +import org.apache.spark.api.java.function.Function; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.io.Serializable; +import java.net.ConnectException; +import java.net.Socket; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +public class JavaReceiverAPISuite implements Serializable { + + @Before + public void setUp() { + System.clearProperty("spark.streaming.clock"); + } + + @After + public void tearDown() { + System.clearProperty("spark.streaming.clock"); + } + + @Test + public void testReceiver() throws InterruptedException { + TestServer server = new TestServer(0); + server.start(); + + final AtomicLong dataCounter = new AtomicLong(0); + + try { + JavaStreamingContext ssc = new JavaStreamingContext("local[2]", "test", new Duration(200)); + JavaReceiverInputDStream input = + ssc.receiverStream(new JavaSocketReceiver("localhost", server.port())); + JavaDStream mapped = input.map(new Function() { + @Override + public String call(String v1) throws Exception { + return v1 + "."; + } + }); + mapped.foreachRDD(new Function, Void>() { + @Override + public Void call(JavaRDD rdd) throws Exception { + long count = rdd.count(); + dataCounter.addAndGet(count); + return null; + } + }); + + ssc.start(); + long startTime = System.currentTimeMillis(); + long timeout = 10000; + + Thread.sleep(200); + for (int i = 0; i < 6; i++) { + server.send("" + i + "\n"); // \n to make sure these are separate lines + Thread.sleep(100); + } + while (dataCounter.get() == 0 && System.currentTimeMillis() - startTime < timeout) { + Thread.sleep(100); + } + ssc.stop(); + assertTrue(dataCounter.get() > 0); + } finally { + server.stop(); + } + } +} + +class JavaSocketReceiver extends Receiver { + + String host = null; + int port = -1; + + public JavaSocketReceiver(String host_ , int port_) { + super(StorageLevel.MEMORY_AND_DISK()); + host = host_; + port = port_; + } + + @Override + public void onStart() { + new Thread() { + @Override public void run() { + receive(); + } + }.start(); + } + + @Override + public void onStop() { + } + + private void receive() { + Socket socket = null; + try { + socket = new Socket(host, port); + BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream())); + String userInput; + while ((userInput = in.readLine()) != null) { + store(userInput); + } + in.close(); + socket.close(); + } catch(ConnectException ce) { + ce.printStackTrace(); + restart("Could not connect", ce); + } catch(Throwable t) { + t.printStackTrace(); + restart("Error receiving data", t); + } + } +} + diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala index 33f6df8f88177..c0ea0491c313d 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala @@ -26,6 +26,7 @@ import org.apache.spark.streaming._ import java.util.ArrayList import collection.JavaConversions._ import org.apache.spark.api.java.JavaRDDLike +import org.apache.spark.streaming.dstream.DStream /** Exposes streaming test functionality in a Java-friendly way. */ trait JavaTestBase extends TestSuiteBase { @@ -51,8 +52,7 @@ trait JavaTestBase extends TestSuiteBase { * [[org.apache.spark.streaming.TestOutputStream]]. **/ def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]( - dstream: JavaDStreamLike[T, This, R]) = - { + dstream: JavaDStreamLike[T, This, R]) = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val ostream = new TestOutputStreamWithPartitions(dstream.dstream) 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 8aec27e39478a..04925886c39e4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -19,14 +19,16 @@ package org.apache.spark.streaming import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.SparkContext._ import util.ManualClock -import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.streaming.dstream.{WindowedDStream, DStream} import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.reflect.ClassTag +import org.apache.spark.storage.StorageLevel +import scala.collection.mutable class BasicOperationsSuite extends TestSuiteBase { test("map") { @@ -396,6 +398,16 @@ class BasicOperationsSuite extends TestSuiteBase { Thread.sleep(1000) } + test("slice - has not been initialized") { + val ssc = new StreamingContext(conf, Seconds(1)) + val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4)) + val stream = new TestInputStream[Int](ssc, input, 2) + val thrown = intercept[SparkException] { + stream.slice(new Time(0), new Time(1000)) + } + assert(thrown.getMessage.contains("has not been initialized")) + } + val cleanupTestInput = (0 until 10).map(x => Seq(x, x + 1)).toSeq test("rdd cleanup - map and window") { @@ -450,6 +462,78 @@ class BasicOperationsSuite extends TestSuiteBase { assert(!stateStream.generatedRDDs.contains(Time(4000))) } + test("rdd cleanup - input blocks and persisted RDDs") { + // Actually receive data over through receiver to create BlockRDDs + + // Start the server + val testServer = new TestServer() + testServer.start() + + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) + val mappedStream = networkStream.map(_ + ".").persist() + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(mappedStream, outputBuffer) + + outputStream.register() + ssc.start() + + // Feed data to the server to send to the network receiver + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val input = Seq(1, 2, 3, 4, 5, 6) + + val blockRdds = new mutable.HashMap[Time, BlockRDD[_]] + val persistentRddIds = new mutable.HashMap[Time, Int] + + def collectRddInfo() { // get all RDD info required for verification + networkStream.generatedRDDs.foreach { case (time, rdd) => + blockRdds(time) = rdd.asInstanceOf[BlockRDD[_]] + } + mappedStream.generatedRDDs.foreach { case (time, rdd) => + persistentRddIds(time) = rdd.id + } + } + + Thread.sleep(200) + for (i <- 0 until input.size) { + testServer.send(input(i).toString + "\n") + Thread.sleep(200) + clock.addToTime(batchDuration.milliseconds) + collectRddInfo() + } + + Thread.sleep(200) + collectRddInfo() + logInfo("Stopping server") + testServer.stop() + logInfo("Stopping context") + + // verify data has been received + assert(outputBuffer.size > 0) + assert(blockRdds.size > 0) + assert(persistentRddIds.size > 0) + + import Time._ + + val latestPersistedRddId = persistentRddIds(persistentRddIds.keySet.max) + val earliestPersistedRddId = persistentRddIds(persistentRddIds.keySet.min) + val latestBlockRdd = blockRdds(blockRdds.keySet.max) + val earliestBlockRdd = blockRdds(blockRdds.keySet.min) + // verify that the latest mapped RDD is persisted but the earliest one has been unpersisted + assert(ssc.sparkContext.persistentRdds.contains(latestPersistedRddId)) + assert(!ssc.sparkContext.persistentRdds.contains(earliestPersistedRddId)) + + // verify that the latest input blocks are present but the earliest blocks have been removed + assert(latestBlockRdd.isValid) + assert(latestBlockRdd.collect != null) + assert(!earliestBlockRdd.isValid) + earliestBlockRdd.blockIds.foreach { blockId => + assert(!ssc.sparkContext.env.blockManager.master.contains(blockId)) + } + ssc.stop() + } + /** Test cleanup of RDDs in DStream metadata */ def runCleanupTest[T: ClassTag]( conf2: SparkConf, 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 952511d411a8e..3fa254065cc44 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -36,15 +36,12 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.NetworkReceiver -import org.apache.spark.streaming.receivers.Receiver import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils +import org.apache.spark.streaming.receiver.{ActorHelper, Receiver} class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { - val testPort = 9999 - test("socket input stream") { // Start the server val testServer = new TestServer() @@ -52,7 +49,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) + val networkStream = ssc.socketTextStream( + "localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(networkStream, outputBuffer) def output = outputBuffer.flatMap(x => x) @@ -144,7 +142,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } - // TODO: This test makes assumptions about Thread.sleep() and is flaky + // TODO: This test works in IntelliJ but not through SBT ignore("actor input stream") { // Start the server val testServer = new TestServer() @@ -207,7 +205,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // set up the network stream using the test receiver val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.networkStream[Int](testReceiver) + val networkStream = ssc.receiverStream[Int](testReceiver) val countStream = networkStream.count val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]] val outputStream = new TestOutputStream(countStream, outputBuffer) @@ -289,19 +287,8 @@ class TestServer(portToBind: Int = 0) extends Logging { def port = serverSocket.getLocalPort } -object TestServer { - def main(args: Array[String]) { - val s = new TestServer() - s.start() - while(true) { - Thread.sleep(1000) - s.send("hello") - } - } -} - /** This is an actor for testing actor input stream */ -class TestActor(port: Int) extends Actor with Receiver { +class TestActor(port: Int) extends Actor with ActorHelper { def bytesToString(byteString: ByteString) = byteString.utf8String @@ -309,24 +296,22 @@ class TestActor(port: Int) extends Actor with Receiver { def receive = { case IO.Read(socket, bytes) => - pushBlock(bytesToString(bytes)) + store(bytesToString(bytes)) } } /** This is a receiver to test multiple threads inserting data using block generator */ class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int) - extends NetworkReceiver[Int] { + extends Receiver[Int](StorageLevel.MEMORY_ONLY_SER) with Logging { lazy val executorPool = Executors.newFixedThreadPool(numThreads) - lazy val blockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY) lazy val finishCount = new AtomicInteger(0) - protected def onStart() { - blockGenerator.start() + def onStart() { (1 to numThreads).map(threadId => { val runnable = new Runnable { def run() { (1 to numRecordsPerThread).foreach(i => - blockGenerator += (threadId * numRecordsPerThread + i) ) + store(threadId * numRecordsPerThread + i) ) if (finishCount.incrementAndGet == numThreads) { MultiThreadTestReceiver.haveAllThreadsFinished = true } @@ -337,7 +322,7 @@ class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int) }) } - protected def onStop() { + def onStop() { executorPool.shutdown() } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala new file mode 100644 index 0000000000000..303d149d285e1 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -0,0 +1,255 @@ +/* + * 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.streaming + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.language.postfixOps + +import org.apache.spark.SparkConf +import org.apache.spark.storage.{StorageLevel, StreamBlockId} +import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver, ReceiverSupervisor} +import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ +import scala.language.postfixOps + +/** Testsuite for testing the network receiver behavior */ +class NetworkReceiverSuite extends FunSuite with Timeouts { + + test("network receiver life cycle") { + + val receiver = new FakeReceiver + val executor = new FakeReceiverSupervisor(receiver) + + assert(executor.isAllEmpty) + + // Thread that runs the executor + val executingThread = new Thread() { + override def run() { + executor.start() + executor.awaitTermination() + } + } + + // Start the receiver + executingThread.start() + + // Verify that the receiver + intercept[Exception] { + failAfter(200 millis) { + executingThread.join() + } + } + + // Verify that receiver was started + assert(receiver.onStartCalled) + assert(executor.isReceiverStarted) + assert(receiver.isStarted) + assert(!receiver.isStopped()) + assert(receiver.otherThread.isAlive) + eventually(timeout(100 millis), interval(10 millis)) { + assert(receiver.receiving) + } + + // Verify whether the data stored by the receiver was sent to the executor + val byteBuffer = ByteBuffer.allocate(100) + val arrayBuffer = new ArrayBuffer[Int]() + val iterator = arrayBuffer.iterator + receiver.store(1) + receiver.store(byteBuffer) + receiver.store(arrayBuffer) + receiver.store(iterator) + assert(executor.singles.size === 1) + assert(executor.singles.head === 1) + assert(executor.byteBuffers.size === 1) + assert(executor.byteBuffers.head.eq(byteBuffer)) + assert(executor.iterators.size === 1) + assert(executor.iterators.head.eq(iterator)) + assert(executor.arrayBuffers.size === 1) + assert(executor.arrayBuffers.head.eq(arrayBuffer)) + + // Verify whether the exceptions reported by the receiver was sent to the executor + val exception = new Exception + receiver.reportError("Error", exception) + assert(executor.errors.size === 1) + assert(executor.errors.head.eq(exception)) + + // Verify restarting actually stops and starts the receiver + receiver.restart("restarting", null, 100) + eventually(timeout(50 millis), interval(10 millis)) { + // receiver will be stopped async + assert(receiver.isStopped) + assert(receiver.onStopCalled) + } + eventually(timeout(1000 millis), interval(100 millis)) { + // receiver will be started async + assert(receiver.onStartCalled) + assert(executor.isReceiverStarted) + assert(receiver.isStarted) + assert(!receiver.isStopped) + assert(receiver.receiving) + } + + // Verify that stopping actually stops the thread + failAfter(100 millis) { + receiver.stop("test") + assert(receiver.isStopped) + assert(!receiver.otherThread.isAlive) + + // The thread that started the executor should complete + // as stop() stops everything + executingThread.join() + } + } + + test("block generator") { + val blockGeneratorListener = new FakeBlockGeneratorListener + val blockInterval = 200 + val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString) + val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) + val expectedBlocks = 5 + val waitTime = expectedBlocks * blockInterval + (blockInterval / 2) + val generatedData = new ArrayBuffer[Int] + + // Generate blocks + val startTime = System.currentTimeMillis() + blockGenerator.start() + var count = 0 + while(System.currentTimeMillis - startTime < waitTime) { + blockGenerator += count + generatedData += count + count += 1 + Thread.sleep(10) + } + blockGenerator.stop() + + val recordedData = blockGeneratorListener.arrayBuffers.flatten + assert(blockGeneratorListener.arrayBuffers.size > 0) + assert(recordedData.toSet === generatedData.toSet) + } + + /** + * An implementation of NetworkReceiver that is used for testing a receiver's life cycle. + */ + class FakeReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) { + var otherThread: Thread = null + var receiving = false + var onStartCalled = false + var onStopCalled = false + + def onStart() { + otherThread = new Thread() { + override def run() { + receiving = true + while(!isStopped()) { + Thread.sleep(10) + } + } + } + onStartCalled = true + otherThread.start() + + } + + def onStop() { + onStopCalled = true + otherThread.join() + } + + def reset() { + receiving = false + onStartCalled = false + onStopCalled = false + } + } + + /** + * An implementation of NetworkReceiverExecutor used for testing a NetworkReceiver. + * Instead of storing the data in the BlockManager, it stores all the data in a local buffer + * that can used for verifying that the data has been forwarded correctly. + */ + class FakeReceiverSupervisor(receiver: FakeReceiver) + extends ReceiverSupervisor(receiver, new SparkConf()) { + val singles = new ArrayBuffer[Any] + val byteBuffers = new ArrayBuffer[ByteBuffer] + val iterators = new ArrayBuffer[Iterator[_]] + val arrayBuffers = new ArrayBuffer[ArrayBuffer[_]] + val errors = new ArrayBuffer[Throwable] + + /** Check if all data structures are clean */ + def isAllEmpty = { + singles.isEmpty && byteBuffers.isEmpty && iterators.isEmpty && + arrayBuffers.isEmpty && errors.isEmpty + } + + def pushSingle(data: Any) { + singles += data + } + + def pushBytes( + bytes: ByteBuffer, + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + byteBuffers += bytes + } + + def pushIterator( + iterator: Iterator[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + iterators += iterator + } + + def pushArrayBuffer( + arrayBuffer: ArrayBuffer[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + arrayBuffers += arrayBuffer + } + + def reportError(message: String, throwable: Throwable) { + errors += throwable + } + } + + /** + * An implementation of BlockGeneratorListener that is used to test the BlockGenerator. + */ + class FakeBlockGeneratorListener(pushDelay: Long = 0) extends BlockGeneratorListener { + // buffer of data received as ArrayBuffers + val arrayBuffers = new ArrayBuffer[ArrayBuffer[Int]] + val errors = new ArrayBuffer[Throwable] + + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) { + val bufferOfInts = arrayBuffer.map(_.asInstanceOf[Int]) + arrayBuffers += bufferOfInts + Thread.sleep(0) + } + + def onError(message: String, throwable: Throwable) { + errors += throwable + } + } +} + diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index ad5367ab941a2..ee0bc8b7d6a71 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -21,7 +21,8 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.{DStream, NetworkReceiver} +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.util.{MetadataCleaner, Utils} import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Timeouts @@ -37,15 +38,10 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val batchDuration = Milliseconds(500) val sparkHome = "someDir" val envPair = "key" -> "value" - val ttl = StreamingContext.DEFAULT_CLEANER_TTL + 100 var sc: SparkContext = null var ssc: StreamingContext = null - before { - System.clearProperty("spark.cleaner.ttl") - } - after { if (ssc != null) { ssc.stop() @@ -61,67 +57,51 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc = new StreamingContext(master, appName, batchDuration) assert(ssc.sparkContext.conf.get("spark.master") === master) assert(ssc.sparkContext.conf.get("spark.app.name") === appName) - assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) } test("from no conf + spark home") { ssc = new StreamingContext(master, appName, batchDuration, sparkHome, Nil) assert(ssc.conf.get("spark.home") === sparkHome) - assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) } test("from no conf + spark home + env") { ssc = new StreamingContext(master, appName, batchDuration, sparkHome, Nil, Map(envPair)) assert(ssc.conf.getExecutorEnv.exists(_ == envPair)) - assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) - } - - test("from conf without ttl set") { - val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - ssc = new StreamingContext(myConf, batchDuration) - assert(MetadataCleaner.getDelaySeconds(ssc.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) } - test("from conf with ttl set") { + test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", ttl.toString) + myConf.set("spark.cleaner.ttl", "10") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === ttl) + assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } - test("from existing SparkContext without ttl set") { + test("from existing SparkContext") { sc = new SparkContext(master, appName) - val exception = intercept[SparkException] { - ssc = new StreamingContext(sc, batchDuration) - } - assert(exception.getMessage.contains("ttl")) + ssc = new StreamingContext(sc, batchDuration) } - test("from existing SparkContext with ttl set") { + test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", ttl.toString) + myConf.set("spark.cleaner.ttl", "10") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === ttl) + assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", ttl.toString) + myConf.set("spark.cleaner.ttl", "10") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) - assert(MetadataCleaner.getDelaySeconds(cp.sparkConf) === ttl) + assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(MetadataCleaner.getDelaySeconds(newCp.sparkConf) === ttl) + assert(newCp.sparkConf.getInt("spark.cleaner.ttl", -1) === 10) ssc = new StreamingContext(null, newCp, null) - assert(MetadataCleaner.getDelaySeconds(ssc.conf) === ttl) + assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } test("start and stop state check") { @@ -181,15 +161,15 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w conf.set("spark.cleaner.ttl", "3600") sc = new SparkContext(conf) for (i <- 1 to 4) { - logInfo("==================================") - ssc = new StreamingContext(sc, batchDuration) + logInfo("==================================\n\n\n") + ssc = new StreamingContext(sc, Milliseconds(100)) var runningCount = 0 TestReceiver.counter.set(1) - val input = ssc.networkStream(new TestReceiver) + val input = ssc.receiverStream(new TestReceiver) input.count.foreachRDD(rdd => { val count = rdd.first() - logInfo("Count = " + count) runningCount += count.toInt + logInfo("Count = " + count + ", Running count = " + runningCount) }) ssc.start() ssc.awaitTermination(500) @@ -216,12 +196,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc.start() } - // test whether waitForStop() exits after give amount of time + // test whether awaitTermination() exits after give amount of time failAfter(1000 millis) { ssc.awaitTermination(500) } - // test whether waitForStop() does not exit if not time is given + // test whether awaitTermination() does not exit if not time is given val exception = intercept[Exception] { failAfter(1000 millis) { ssc.awaitTermination() @@ -276,23 +256,26 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w class TestException(msg: String) extends Exception(msg) /** Custom receiver for testing whether all data received by a receiver gets processed or not */ -class TestReceiver extends NetworkReceiver[Int] { - protected lazy val blockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY) - protected def onStart() { - blockGenerator.start() - logInfo("BlockGenerator started on thread " + receivingThread) - try { - while(true) { - blockGenerator += TestReceiver.counter.getAndIncrement - Thread.sleep(0) +class TestReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging { + + var receivingThreadOption: Option[Thread] = None + + def onStart() { + val thread = new Thread() { + override def run() { + logInfo("Receiving started") + while (!isStopped) { + store(TestReceiver.counter.getAndIncrement) + } + logInfo("Receiving stopped at count value of " + TestReceiver.counter.get()) } - } finally { - logInfo("Receiving stopped at count value of " + TestReceiver.counter.get()) } + receivingThreadOption = Some(thread) + thread.start() } - protected def onStop() { - blockGenerator.stop() + def onStop() { + // no cleanup to be done, the receiving thread should stop on it own } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 9e0f2c900e8ba..ef0efa552ceaf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -17,10 +17,20 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.scheduler._ import scala.collection.mutable.ArrayBuffer -import org.scalatest.matchers.ShouldMatchers +import scala.concurrent.Future +import scala.concurrent.ExecutionContext.Implicits.global +import scala.language.postfixOps + +import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler._ + +import org.scalatest.matchers.ShouldMatchers +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ +import org.apache.spark.Logging class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { @@ -32,7 +42,7 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { override def batchDuration = Milliseconds(100) override def actuallyWait = true - test("basic BatchInfo generation") { + test("batch info reporting") { val ssc = setupStreams(input, operation) val collector = new BatchInfoCollector ssc.addStreamingListener(collector) @@ -54,6 +64,31 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { isInIncreasingOrder(batchInfos.map(_.processingEndTime.get)) should be (true) } + test("receiver info reporting") { + val ssc = new StreamingContext("local[2]", "test", Milliseconds(1000)) + val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver) + inputStream.foreachRDD(_.count) + + val collector = new ReceiverInfoCollector + ssc.addStreamingListener(collector) + + ssc.start() + try { + eventually(timeout(1000 millis), interval(20 millis)) { + collector.startedReceiverStreamIds.size should be >= 1 + collector.startedReceiverStreamIds(0) should equal (0) + collector.stoppedReceiverStreamIds should have size 1 + collector.stoppedReceiverStreamIds(0) should equal (0) + collector.receiverErrors should have size 1 + collector.receiverErrors(0)._1 should equal (0) + collector.receiverErrors(0)._2 should include ("report error") + collector.receiverErrors(0)._3 should include ("report exception") + } + } finally { + ssc.stop() + } + } + /** Check if a sequence of numbers is in increasing order */ def isInIncreasingOrder(seq: Seq[Long]): Boolean = { for(i <- 1 until seq.size) { @@ -61,12 +96,47 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { } true } +} + +/** Listener that collects information on processed batches */ +class BatchInfoCollector extends StreamingListener { + val batchInfos = new ArrayBuffer[BatchInfo] + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { + batchInfos += batchCompleted.batchInfo + } +} + +/** Listener that collects information on processed batches */ +class ReceiverInfoCollector extends StreamingListener { + val startedReceiverStreamIds = new ArrayBuffer[Int] + val stoppedReceiverStreamIds = new ArrayBuffer[Int]() + val receiverErrors = new ArrayBuffer[(Int, String, String)]() + + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { + startedReceiverStreamIds += receiverStarted.receiverInfo.streamId + } + + override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { + stoppedReceiverStreamIds += receiverStopped.receiverInfo.streamId + } + + override def onReceiverError(receiverError: StreamingListenerReceiverError) { + receiverErrors += ((receiverError.receiverInfo.streamId, + receiverError.receiverInfo.lastErrorMessage, receiverError.receiverInfo.lastError)) + } +} - /** Listener that collects information on processed batches */ - class BatchInfoCollector extends StreamingListener { - val batchInfos = new ArrayBuffer[BatchInfo] - override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { - batchInfos += batchCompleted.batchInfo +class StreamingListenerSuiteReceiver extends Receiver[Any](StorageLevel.MEMORY_ONLY) with Logging { + def onStart() { + Future { + logInfo("Started receiver and sleeping") + Thread.sleep(10) + logInfo("Reporting error and sleeping") + reportError("test report error", new Exception("test report exception")) + Thread.sleep(10) + logInfo("Stopping") + stop("test stop error") } } + def onStop() { } } 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 aa2d5c2fc2454..8036f77c973ae 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -137,7 +137,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { val conf = new SparkConf() .setMaster(master) .setAppName(framework) - .set("spark.cleaner.ttl", StreamingContext.DEFAULT_CLEANER_TTL.toString) // 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 { } ) @@ -156,6 +155,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { def afterFunction() { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.streaming.clock") } before(beforeFunction) diff --git a/tools/pom.xml b/tools/pom.xml index ae2ba64e07c21..1875c497bc61c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -30,20 +30,6 @@ Spark Project Tools http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index 3fb85e1ff73a8..a433e8e2e89f4 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -41,7 +41,7 @@ object GenerateMIMAIgnore { private def classesPrivateWithin(packageName: String): Set[String] = { - val classes = getClasses(packageName, classLoader) + val classes = getClasses(packageName) val privateClasses = mutable.HashSet[String]() def isPackagePrivate(className: String) = { @@ -50,7 +50,7 @@ object GenerateMIMAIgnore { is a module or class. */ val privateAsClass = mirror - .staticClass(className) + .classSymbol(Class.forName(className, false, classLoader)) .privateWithin .fullName .startsWith(packageName) @@ -108,8 +108,7 @@ object GenerateMIMAIgnore { * Scans all classes accessible from the context class loader which belong to the given package * and subpackages both from directories and jars present on the classpath. */ - private def getClasses(packageName: String, - classLoader: ClassLoader = Thread.currentThread().getContextClassLoader): Set[String] = { + private def getClasses(packageName: String): Set[String] = { val path = packageName.replace('.', '/') val resources = classLoader.getResources(path) @@ -128,7 +127,7 @@ object GenerateMIMAIgnore { val jar = new JarFile(new File(jarPath)) val enums = jar.entries().map(_.getName).filter(_.startsWith(packageName)) val classes = for (entry <- enums if entry.endsWith(".class")) - yield Class.forName(entry.replace('/', '.').stripSuffix(".class")) + yield Class.forName(entry.replace('/', '.').stripSuffix(".class"), false, classLoader) classes } } diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index d0aeaceb0d23c..e076ca1d44b97 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,20 +24,6 @@ ../pom.xml - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark spark-yarn-alpha_2.10 jar diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index f078d06aafad0..fc13dbecb4555 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -39,6 +39,9 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils +/** + * An application master that runs the users driver program and allocates executors. + */ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { @@ -144,12 +147,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) .orElse(Option(System.getenv("LOCAL_DIRS"))) - + localDirs match { case None => throw new Exception("Yarn Local dirs can't be empty") case Some(l) => l } - } + } private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() @@ -318,8 +321,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, logInfo("Allocating %d containers to make up for (potentially) lost containers". format(missingExecutorCount)) yarnAllocator.allocateContainers(missingExecutorCount) + } else { + sendProgress() } - else sendProgress() Thread.sleep(sleepTime) } } @@ -358,7 +362,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, return } isFinished = true - + logInfo("finishApplicationMaster with " + status) if (registered) { val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 0179b0600c61f..8226207de42b8 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -33,7 +33,9 @@ import org.apache.hadoop.yarn.util.{Apps, Records} import org.apache.spark.{Logging, SparkConf} - +/** + * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. + */ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { @@ -167,8 +169,10 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa object Client { def main(argStrings: Array[String]) { - println("WARNING: This client is deprecated and will be removed in a future version of Spark.") - println("Use ./bin/spark-submit with \"--master yarn\"") + if (!sys.props.contains("SPARK_SUBMIT")) { + println("WARNING: This client is deprecated and will be removed in a " + + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") + } // Set an env variable indicating we are running in YARN mode. // Note that anything with SPARK prefix gets propagated to all (remote) processes diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 21f14576efe8a..65b7215afbd4c 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -34,6 +34,12 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo +/** + * An application master that allocates executors on behalf of a driver that is running outside + * the cluster. + * + * This is used only in yarn-client mode. + */ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { @@ -237,8 +243,9 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp logInfo("Allocating " + missingExecutorCount + " containers to make up for (potentially ?) lost containers") yarnAllocator.allocateContainers(missingExecutorCount) + } else { + sendProgress() } - else sendProgress() Thread.sleep(sleepTime) } } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index d6d46a5f6ce42..856391e52b2df 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -54,14 +54,18 @@ object AllocationType extends Enumeration { // Note that right now, we assume all node asks as uniform in terms of capabilities and priority // Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for // more info on how we are requesting for containers. + +/** + * Acquires resources for executors from a ResourceManager and launches executors in new containers. + */ private[yarn] class YarnAllocationHandler( val conf: Configuration, - val resourceManager: AMRMProtocol, + val resourceManager: AMRMProtocol, val appAttemptId: ApplicationAttemptId, val maxExecutors: Int, val executorMemory: Int, val executorCores: Int, - val preferredHostToCount: Map[String, Int], + val preferredHostToCount: Map[String, Int], val preferredRackToCount: Map[String, Int], val sparkConf: SparkConf) extends Logging { @@ -132,9 +136,10 @@ private[yarn] class YarnAllocationHandler( val containers = hostToContainers.getOrElseUpdate(host, new ArrayBuffer[Container]()) containers += container + } else { + // Add all ignored containers to released list + releasedContainerList.add(container.getId()) } - // Add all ignored containers to released list - else releasedContainerList.add(container.getId()) } // Find the appropriate containers to use. Slightly non trivial groupBy ... @@ -155,8 +160,7 @@ private[yarn] class YarnAllocationHandler( dataLocalContainers.put(candidateHost, remainingContainers) // all consumed remainingContainers = null - } - else if (requiredHostCount > 0) { + } else if (requiredHostCount > 0) { // Container list has more containers than we need for data locality. // Split into two : data local container count of (remainingContainers.size - // requiredHostCount) and rest as remainingContainer @@ -166,7 +170,7 @@ private[yarn] class YarnAllocationHandler( // remainingContainers = remaining // yarn has nasty habit of allocating a tonne of containers on a host - discourage this : - // add remaining to release list. If we have insufficient containers, next allocation + // add remaining to release list. If we have insufficient containers, next allocation // cycle will reallocate (but wont treat it as data local) for (container <- remaining) releasedContainerList.add(container.getId()) remainingContainers = null @@ -178,7 +182,7 @@ private[yarn] class YarnAllocationHandler( if (rack != null){ val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) - val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - + val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - rackLocalContainers.get(rack).getOrElse(List()).size @@ -187,8 +191,7 @@ private[yarn] class YarnAllocationHandler( dataLocalContainers.put(rack, remainingContainers) // All consumed remainingContainers = null - } - else if (requiredRackCount > 0) { + } else if (requiredRackCount > 0) { // container list has more containers than we need for data locality. // Split into two : data local container count of (remainingContainers.size - // requiredRackCount) and rest as remainingContainer @@ -209,7 +212,7 @@ private[yarn] class YarnAllocationHandler( } } - // Now that we have split the containers into various groups, go through them in order : + // Now that we have split the containers into various groups, go through them in order : // first host local, then rack local and then off rack (everything else). // Note that the list we create below tries to ensure that not all containers end up within a // host if there are sufficiently large number of hosts/containers. @@ -234,8 +237,7 @@ private[yarn] class YarnAllocationHandler( releasedContainerList.add(containerId) // reset counter back to old value. numExecutorsRunning.decrementAndGet() - } - else { + } else { // Deallocate + allocate can result in reusing id's wrongly - so use a different counter // (executorIdCounter) val executorId = executorIdCounter.incrementAndGet().toString @@ -289,8 +291,7 @@ private[yarn] class YarnAllocationHandler( // Was this released by us ? If yes, then simply remove from containerSet and move on. if (pendingReleaseContainers.containsKey(containerId)) { pendingReleaseContainers.remove(containerId) - } - else { + } else { // Simply decrement count - next iteration of ReporterThread will take care of allocating. numExecutorsRunning.decrementAndGet() logInfo("Completed container %s (state: %s, exit status: %s)".format( @@ -315,8 +316,11 @@ private[yarn] class YarnAllocationHandler( assert (containerSet != null) containerSet -= containerId - if (containerSet.isEmpty) allocatedHostToContainersMap.remove(host) - else allocatedHostToContainersMap.update(host, containerSet) + if (containerSet.isEmpty) { + allocatedHostToContainersMap.remove(host) + } else { + allocatedHostToContainersMap.update(host, containerSet) + } allocatedContainerToHostMap -= containerId @@ -324,8 +328,11 @@ private[yarn] class YarnAllocationHandler( val rack = YarnAllocationHandler.lookupRack(conf, host) if (rack != null) { val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 - if (rackCount > 0) allocatedRackCount.put(rack, rackCount) - else allocatedRackCount.remove(rack) + if (rackCount > 0) { + allocatedRackCount.put(rack, rackCount) + } else { + allocatedRackCount.remove(rack) + } } } } @@ -361,10 +368,10 @@ private[yarn] class YarnAllocationHandler( } } - val requestedContainers: ArrayBuffer[ResourceRequest] = + val requestedContainers: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](rackToCounts.size) for ((rack, count) <- rackToCounts){ - requestedContainers += + requestedContainers += createResourceRequest(AllocationType.RACK, rack, count, YarnAllocationHandler.PRIORITY) } @@ -397,11 +404,10 @@ private[yarn] class YarnAllocationHandler( preferredHostToCount.isEmpty) resourceRequests = List(createResourceRequest( AllocationType.ANY, null, numExecutors, YarnAllocationHandler.PRIORITY)) - } - else { - // request for all hosts in preferred nodes and for numExecutors - + } else { + // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. - val hostContainerRequests: ArrayBuffer[ResourceRequest] = + val hostContainerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](preferredHostToCount.size) for ((candidateHost, candidateCount) <- preferredHostToCount) { val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) @@ -445,8 +451,7 @@ private[yarn] class YarnAllocationHandler( if (numExecutors > 0) { logInfo("Allocating %d executor containers with %d of memory each.".format(numExecutors, executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) - } - else { + } else { logDebug("Empty allocation req .. release : " + releasedContainerList) } @@ -463,7 +468,7 @@ private[yarn] class YarnAllocationHandler( private def createResourceRequest( - requestType: AllocationType.AllocationType, + requestType: AllocationType.AllocationType, resource:String, numExecutors: Int, priority: Int): ResourceRequest = { @@ -524,7 +529,7 @@ private[yarn] class YarnAllocationHandler( if (! retval.isEmpty) { releasedContainerList.removeAll(retval) for (v <- retval) pendingReleaseContainers.put(v, true) - logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + + logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + pendingReleaseContainers) } @@ -535,7 +540,7 @@ private[yarn] class YarnAllocationHandler( object YarnAllocationHandler { val ANY_HOST = "*" - // All requests are issued with same priority : we do not (yet) have any distinction between + // All requests are issued with same priority : we do not (yet) have any distinction between // request types (like map/reduce in hadoop for example) val PRIORITY = 1 @@ -544,7 +549,7 @@ object YarnAllocationHandler { // Host to rack map - saved from allocation requests // We are expecting this not to change. - // Note that it is possible for this to change : and RM will indicate that to us via update + // Note that it is possible for this to change : and RM will indicate that to us via update // response to allocate. But we are punting on handling that for now. private val hostToRack = new ConcurrentHashMap[String, String]() private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() @@ -561,7 +566,7 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numExecutors, + args.numExecutors, args.executorMemory, args.executorCores, Map[String, Int](), @@ -583,7 +588,7 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numExecutors, + args.numExecutors, args.executorMemory, args.executorCores, hostToCount, diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 566de712fc280..f2be8217a2f8a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -22,16 +22,13 @@ import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map +import scala.collection.mutable.{HashMap, ListBuffer, Map} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ @@ -39,19 +36,18 @@ 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.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Records, Apps} - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.Utils -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment - +import org.apache.hadoop.yarn.util.{Apps, Records} +import org.apache.spark.{Logging, SparkConf, SparkContext} /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, - * which will launch a Spark master process and negotiate resources throughout its duration. + * Client submits an application to the YARN ResourceManager. + * + * Depending on the deployment mode this will launch one of two application master classes: + * 1. In standalone mode, it will launch an [[org.apache.spark.deploy.yarn.ApplicationMaster]] + * which launches a driver program inside of the cluster. + * 2. In client mode, it will launch an [[org.apache.spark.deploy.yarn.ExecutorLauncher]] to + * request executors on behalf of a driver running outside of the cluster. */ trait ClientBase extends Logging { val args: ClientArguments @@ -63,14 +59,15 @@ trait ClientBase extends Logging { private val distCacheMgr = new ClientDistributedCacheManager() // Staging directory is private! -> rwx-------- - val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short) + 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(0644: Short) + val APP_FILE_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { Map( - (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", ((args.userJar == null && args.amClass == classOf[ApplicationMaster].getName) -> "Error: You must specify a user jar when running in standalone mode!"), (args.userClass == null) -> "Error: You must specify a user class!", @@ -208,7 +205,7 @@ trait ClientBase extends Logging { val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() Map( - ClientBase.SPARK_JAR -> System.getenv("SPARK_JAR"), ClientBase.APP_JAR -> args.userJar, + ClientBase.SPARK_JAR -> ClientBase.getSparkJar, ClientBase.APP_JAR -> args.userJar, ClientBase.LOG4J_PROP -> System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) ).foreach { case(destName, _localPath) => val localPath: String = if (_localPath != null) _localPath.trim() else "" @@ -251,8 +248,10 @@ trait ClientBase extends Logging { logInfo("Setting up the launch environment") val env = new HashMap[String, String]() + + val extraCp = sparkConf.getOption("spark.driver.extraClassPath") val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) - ClientBase.populateClasspath(args, yarnConf, sparkConf, log4jConf, env) + ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -268,9 +267,6 @@ trait ClientBase extends Logging { YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), File.pathSeparator) - // Add each SPARK_* key to the environment. - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } - env } @@ -299,13 +295,13 @@ trait ClientBase extends Logging { val amMemory = calculateAMMemory(newApp) - var JAVA_OPTS = "" + val JAVA_OPTS = ListBuffer[String]() // Add Xmx for AM memory JAVA_OPTS += "-Xmx" + amMemory + "m" val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) - JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir + JAVA_OPTS += "-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 - @@ -319,35 +315,48 @@ trait ClientBase extends Logging { if (useConcurrentAndIncrementalGC) { // In our expts, using (default) throughput collector has severe perf ramifications in // multi-tenant machines - JAVA_OPTS += " -XX:+UseConcMarkSweepGC " - JAVA_OPTS += " -XX:+CMSIncrementalMode " - JAVA_OPTS += " -XX:+CMSIncrementalPacing " - JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " - JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + JAVA_OPTS += "-XX:+UseConcMarkSweepGC" + JAVA_OPTS += "-XX:+CMSIncrementalMode" + JAVA_OPTS += "-XX:+CMSIncrementalPacing" + JAVA_OPTS += "-XX:CMSIncrementalDutyCycleMin=0" + JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10" } - if (env.isDefinedAt("SPARK_JAVA_OPTS")) { - JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") + // 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. + if (args.amClass == classOf[ExecutorLauncher].getName) { + // If we are being launched in client mode, forward the spark-conf options + // onto the executor launcher + for ((k, v) <- sparkConf.getAll) { + JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" + } + } else { + // If we are being launched in standalone mode, capture and forward any spark + // system properties (e.g. set by spark-class). + for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { + JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" + } + sys.props.get("spark.driver.extraJavaOptions").foreach(opts => JAVA_OPTS += opts) + sys.props.get("spark.driver.libraryPath").foreach(p => JAVA_OPTS += s"-Djava.library.path=$p") } JAVA_OPTS += ClientBase.getLog4jConfiguration(localResources) // Command for the ApplicationMaster - val commands = List[String]( - Environment.JAVA_HOME.$() + "/bin/java" + - " -server " + - JAVA_OPTS + - " " + args.amClass + - " --class " + args.userClass + - " --jar " + args.userJar + - userArgsToString(args) + - " --executor-memory " + args.executorMemory + - " --executor-cores " + args.executorCores + - " --num-executors " + args.numExecutors + - " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + - " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - - logInfo("Command for starting the Spark ApplicationMaster: " + commands(0)) - amContainer.setCommands(commands) + val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ + JAVA_OPTS ++ + Seq(args.amClass, "--class", args.userClass, "--jar ", args.userJar, + userArgsToString(args), + "--executor-memory", args.executorMemory.toString, + "--executor-cores", args.executorCores.toString, + "--num-executors ", args.numExecutors.toString, + "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", + "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + logInfo("Command for starting the Spark ApplicationMaster: " + commands) + + // 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) setupSecurityToken(amContainer) amContainer @@ -361,14 +370,18 @@ object ClientBase { val LOG4J_CONF_ENV_KEY: String = "SPARK_LOG4J_CONF" val LOCAL_SCHEME = "local" + def getSparkJar = sys.env.get("SPARK_JAR").getOrElse(SparkContext.jarOfClass(this.getClass).head) + // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { val classpathEntries = Option(conf.getStrings( YarnConfiguration.YARN_APPLICATION_CLASSPATH)).getOrElse( getDefaultYarnApplicationClasspath()) - for (c <- classpathEntries) { - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, c.trim, - File.pathSeparator) + if (classpathEntries != null) { + for (c <- classpathEntries) { + YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, c.trim, + File.pathSeparator) + } } val mrClasspathEntries = Option(conf.getStrings( @@ -433,10 +446,9 @@ object ClientBase { " -Dlog4j.configuration=" + log4jConf } - def populateClasspath(args: ClientArguments, conf: Configuration, sparkConf: SparkConf, - log4jConf: String, env: HashMap[String, String]) { - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$(), - File.pathSeparator) + def populateClasspath(conf: Configuration, sparkConf: SparkConf, log4jConf: String, + env: HashMap[String, String], extraClassPath: Option[String] = None) { + if (log4jConf != null) { // If a custom log4j config file is provided as a local: URI, add its parent directory to the // classpath. Note that this only works if the custom config's file name is @@ -448,19 +460,27 @@ object ClientBase { File.pathSeparator) } } + + /** Add entry to the classpath. */ + def addClasspathEntry(path: String) = YarnSparkHadoopUtil.addToEnvironment(env, + Environment.CLASSPATH.name, path, File.pathSeparator) + /** Add entry to the classpath. Interpreted as a path relative to the working directory. */ + def addPwdClasspathEntry(entry: String) = addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry) + + extraClassPath.foreach(addClasspathEntry) + + addClasspathEntry(Environment.PWD.$()) // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false") - .toBoolean - if (userClasspathFirst) { - addUserClasspath(args, env) - } - addClasspathEntry(System.getenv("SPARK_JAR"), SPARK_JAR, env); - ClientBase.populateHadoopClasspath(conf, env) - if (!userClasspathFirst) { - addUserClasspath(args, env) + if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { + addPwdClasspathEntry(APP_JAR) + addPwdClasspathEntry(SPARK_JAR) + ClientBase.populateHadoopClasspath(conf, env) + } else { + addPwdClasspathEntry(SPARK_JAR) + ClientBase.populateHadoopClasspath(conf, env) + addPwdClasspathEntry(APP_JAR) } - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, - Environment.PWD.$() + Path.SEPARATOR + "*", File.pathSeparator) + addPwdClasspathEntry("*") } /** diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 40b38661f794d..7d07f6f68046a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -19,26 +19,18 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URI -import java.nio.ByteBuffer -import java.security.PrivilegedExceptionAction import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -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.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} - -import org.apache.spark.{SparkConf, Logging} import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} +import org.apache.spark.{Logging, SparkConf} trait ExecutorRunnableUtil extends Logging { @@ -58,8 +50,10 @@ trait ExecutorRunnableUtil extends Logging { // Set the JVM memory val executorMemoryString = executorMemory + "m" JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " - if (env.isDefinedAt("SPARK_JAVA_OPTS")) { - JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " + + // Set extra Java options for the executor, if defined + sys.props.get("spark.executor.extraJavaOptions").foreach { opts => + JAVA_OPTS += opts } JAVA_OPTS += " -Djava.io.tmpdir=" + @@ -162,8 +156,9 @@ trait ExecutorRunnableUtil extends Logging { def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() + val extraCp = sparkConf.getOption("spark.executor.extraClassPath") val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) - ClientBase.populateClasspath(null, yarnConf, sparkConf, log4jConf, env) + ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp) if (log4jConf != null) { env(ClientBase.LOG4J_CONF_ENV_KEY) = log4jConf } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 832d45b3ad10e..718cb19f57261 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -64,7 +64,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { override def addSecretKeyToUserCredentials(key: String, secret: String) { val creds = new Credentials() - creds.addSecretKey(new Text(key), secret.getBytes()) + creds.addSecretKey(new Text(key), secret.getBytes("utf-8")) addCurrentUserCredentials(creds) } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 161918859e7c4..ce2dde0631ed9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments} +import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -54,7 +54,7 @@ private[spark] class YarnClientSchedulerBackend( "--class", "notused", "--jar", null, "--args", hostport, - "--am-class", "org.apache.spark.deploy.yarn.ExecutorLauncher" + "--am-class", classOf[ExecutorLauncher].getName ) // process any optional arguments, given either as environment variables diff --git a/yarn/pom.xml b/yarn/pom.xml index 3342cb65edcd1..02f36627431b9 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -70,15 +70,6 @@ alpha - - - - - org.apache.avro - avro - - diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index e7915d12aef63..0780f251b595c 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,20 +24,6 @@ ../pom.xml - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark spark-yarn_2.10 jar diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index b225be6a79c0e..90e807160d4b6 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -42,6 +42,9 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils +/** + * An application master that runs the user's driver program and allocates executors. + */ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 77eb1276a0c4e..1b6bfb42a5c1c 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -34,9 +34,7 @@ import org.apache.spark.{Logging, SparkConf} /** - * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, - * which will launch a Spark master process and negotiate resources throughout its duration. + * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's stable API. */ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { @@ -173,8 +171,10 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa object Client { def main(argStrings: Array[String]) { - println("WARNING: This client is deprecated and will be removed in a future version of Spark.") - println("Use ./bin/spark-submit with \"--master yarn\"") + if (!sys.props.contains("SPARK_SUBMIT")) { + println("WARNING: This client is deprecated and will be removed in a " + + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") + } // Set an env variable indicating we are running in YARN mode. // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 67ed591c78bf9..a14bb377aa133 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -35,6 +35,12 @@ import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +/** + * An application master that allocates executors on behalf of a driver that is running outside + * the cluster. + * + * This is used only in yarn-client mode. + */ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 4fafae1aff26f..a979fe4d62630 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -56,6 +56,10 @@ object AllocationType extends Enumeration { // Note that right now, we assume all node asks as uniform in terms of capabilities and priority // Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for // more info on how we are requesting for containers. + +/** + * Acquires resources for executors from a ResourceManager and launches executors in new containers. + */ private[yarn] class YarnAllocationHandler( val conf: Configuration, val amClient: AMRMClient[ContainerRequest],