From 071ff7efa17a30db57a54f517a3cfecf3463db9f Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 20 Jun 2013 17:53:23 -0700 Subject: [PATCH 001/221] Enable building a fat jar for the Spark REPL --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0ea23b446f..e3892371e0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -197,7 +197,7 @@ object SparkBuild extends Build { def replSettings = sharedSettings ++ Seq( name := "spark-repl", libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - ) + ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", From 4cda8f865a003ab354890c4915ea1b5a7674f5b0 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 15:14:48 -0700 Subject: [PATCH 002/221] Add simple usage to start-slave script --- bin/start-slave.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 26b5b9d462..1082c09eb1 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -1,4 +1,7 @@ #!/usr/bin/env bash +# +# Usage: start-slave.sh +# where is like "spark://localhost:7077" bin=`dirname "$0"` bin=`cd "$bin"; pwd` From 0bcaf036050c3d2b4389339927239e0e35bf02ff Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 15:39:52 -0700 Subject: [PATCH 003/221] Split out source distro CLASSPATH logic to a separate script --- run | 123 +++++++------------------------------------ set-dev-classpath.sh | 112 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 131 insertions(+), 104 deletions(-) create mode 100644 set-dev-classpath.sh diff --git a/run b/run index c0065c53f1..30a2885a4d 100755 --- a/run +++ b/run @@ -1,7 +1,5 @@ #!/bin/bash -SCALA_VERSION=2.9.3 - # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" @@ -46,36 +44,6 @@ case "$1" in ;; esac -if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then - if [ "$SCALA_HOME" ]; then - RUNNER="${SCALA_HOME}/bin/scala" - else - if [ `command -v scala` ]; then - RUNNER="scala" - else - echo "SCALA_HOME is not set and scala is not in PATH" >&2 - exit 1 - fi - fi -else - if [ `command -v java` ]; then - RUNNER="java" - else - if [ -z "$JAVA_HOME" ]; then - echo "JAVA_HOME is not set" >&2 - exit 1 - fi - RUNNER="${JAVA_HOME}/bin/java" - fi - if [ -z "$SCALA_LIBRARY_PATH" ]; then - if [ -z "$SCALA_HOME" ]; then - echo "SCALA_HOME is not set" >&2 - exit 1 - fi - SCALA_LIBRARY_PATH="$SCALA_HOME/lib" - fi -fi - # Figure out how much memory to use per executor and set it as an environment # variable so that our process sees it and can report it to Mesos if [ -z "$SPARK_MEM" ] ; then @@ -93,64 +61,28 @@ if [ -e $FWDIR/conf/java-opts ] ; then fi export JAVA_OPTS -CORE_DIR="$FWDIR/core" -REPL_DIR="$FWDIR/repl" -REPL_BIN_DIR="$FWDIR/repl-bin" -EXAMPLES_DIR="$FWDIR/examples" -BAGEL_DIR="$FWDIR/bagel" -STREAMING_DIR="$FWDIR/streaming" -PYSPARK_DIR="$FWDIR/python" +# Check if this is a binary distribution or source distribution +# and build up the classpath appropriately +if [ -f "$FWDIR/RELEASE" ]; then + echo "This is a binary distribution" -# Exit if the user hasn't compiled Spark -if [ ! -e "$CORE_DIR/target" ]; then - echo "Failed to find Spark classes in $CORE_DIR/target" >&2 - echo "You need to compile Spark before running this program" >&2 - exit 1 -fi + if [ `command -v java` ]; then + RUNNER="java" + else + if [ -z "$JAVA_HOME" ]; then + echo "JAVA_HOME is not set" >&2 + exit 1 + fi + RUNNER="${JAVA_HOME}/bin/java" + fi -if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then - echo "Failed to find Spark classes in $REPL_DIR/target" >&2 - echo "You need to compile Spark repl module before running this program" >&2 - exit 1 -fi + CLASSPATH="$SPARK_CLASSPATH:$FWDIR/jars/*" -# Build up classpath -CLASSPATH="$SPARK_CLASSPATH" -CLASSPATH="$CLASSPATH:$FWDIR/conf" -CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" -if [ -n "$SPARK_TESTING" ] ; then - CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" -fi -CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" -CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar -if [ -e "$FWDIR/lib_managed" ]; then - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" -fi -CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" -if [ -e $REPL_BIN_DIR/target ]; then - for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do - CLASSPATH="$CLASSPATH:$jar" - done -fi -CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" -for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do - CLASSPATH="$CLASSPATH:$jar" -done - -# 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. -if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then - # Use the JAR from the SBT build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` -fi -if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then - # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` + # The JVM doesn't read JAVA_OPTS by default so we need to pass it in + EXTRA_ARGS="$JAVA_OPTS" +else + echo "This is a source distribution" + . "$FWDIR/set-dev-classpath.sh" fi # Add hadoop conf dir - else FileSystem.*, etc fail ! @@ -163,22 +95,5 @@ if [ "x" != "x$YARN_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" fi - -# Figure out whether to run our class with java or with the scala launcher. -# In most cases, we'd prefer to execute our process with java because scala -# creates a shell script as the parent of its Java process, which makes it -# hard to kill the child with stuff like Process.destroy(). However, for -# the Spark shell, the wrapper is necessary to properly reset the terminal -# when we exit, so we allow it to set a variable to launch with scala. -if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then - EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS -else - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" - # The JVM doesn't read JAVA_OPTS by default so we need to pass it in - EXTRA_ARGS="$JAVA_OPTS" -fi - export CLASSPATH # Needed for spark-shell exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" diff --git a/set-dev-classpath.sh b/set-dev-classpath.sh new file mode 100644 index 0000000000..4d09bd4416 --- /dev/null +++ b/set-dev-classpath.sh @@ -0,0 +1,112 @@ +# A BASH script to set the classpath for running Spark out of the developer/github tree + +SCALA_VERSION=2.9.3 + +# Figure out where the Scala framework is installed +FWDIR="$(cd `dirname $0`; pwd)" + +if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then + if [ "$SCALA_HOME" ]; then + RUNNER="${SCALA_HOME}/bin/scala" + else + if [ `command -v scala` ]; then + RUNNER="scala" + else + echo "SCALA_HOME is not set and scala is not in PATH" >&2 + exit 1 + fi + fi +else + if [ `command -v java` ]; then + RUNNER="java" + else + if [ -z "$JAVA_HOME" ]; then + echo "JAVA_HOME is not set" >&2 + exit 1 + fi + RUNNER="${JAVA_HOME}/bin/java" + fi + if [ -z "$SCALA_LIBRARY_PATH" ]; then + if [ -z "$SCALA_HOME" ]; then + echo "SCALA_HOME is not set" >&2 + exit 1 + fi + SCALA_LIBRARY_PATH="$SCALA_HOME/lib" + fi +fi + +CORE_DIR="$FWDIR/core" +REPL_DIR="$FWDIR/repl" +REPL_BIN_DIR="$FWDIR/repl-bin" +EXAMPLES_DIR="$FWDIR/examples" +BAGEL_DIR="$FWDIR/bagel" +STREAMING_DIR="$FWDIR/streaming" +PYSPARK_DIR="$FWDIR/python" + +# Exit if the user hasn't compiled Spark +if [ ! -e "$CORE_DIR/target" ]; then + echo "Failed to find Spark classes in $CORE_DIR/target" >&2 + echo "You need to compile Spark before running this program" >&2 + exit 1 +fi + +if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then + echo "Failed to find Spark classes in $REPL_DIR/target" >&2 + echo "You need to compile Spark repl module before running this program" >&2 + exit 1 +fi + +# Build up classpath +CLASSPATH="$SPARK_CLASSPATH" +CLASSPATH="$CLASSPATH:$FWDIR/conf" +CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" +if [ -n "$SPARK_TESTING" ] ; then + CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" +fi +CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" +CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar +if [ -e "$FWDIR/lib_managed" ]; then + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" +fi +CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" +if [ -e $REPL_BIN_DIR/target ]; then + for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do + CLASSPATH="$CLASSPATH:$jar" + done +fi +CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do + CLASSPATH="$CLASSPATH:$jar" +done + +# 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. +if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` +fi +if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then + # Use the JAR from the Maven build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` +fi + +# Figure out whether to run our class with java or with the scala launcher. +# In most cases, we'd prefer to execute our process with java because scala +# creates a shell script as the parent of its Java process, which makes it +# hard to kill the child with stuff like Process.destroy(). However, for +# the Spark shell, the wrapper is necessary to properly reset the terminal +# when we exit, so we allow it to set a variable to launch with scala. +if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then + EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS +else + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" + # The JVM doesn't read JAVA_OPTS by default so we need to pass it in + EXTRA_ARGS="$JAVA_OPTS" +fi From 81df20e5b44407c8c321471be2faee4bf6853fd6 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 17:05:37 -0700 Subject: [PATCH 004/221] Script to create binary distribution for Spark --- make-distribution.sh | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100755 make-distribution.sh diff --git a/make-distribution.sh b/make-distribution.sh new file mode 100755 index 0000000000..34f6d4a36b --- /dev/null +++ b/make-distribution.sh @@ -0,0 +1,30 @@ +#!/bin/bash +# +# Script to create a binary distribution for easy deploys of Spark. +# The distribution directory defaults to dist/ but can be overridden below. +# The distribution contains fat (assembly) jars that include the Scala library, +# so it is completely self contained. + +# Figure out where the Spark framework is installed +FWDIR="$(cd `dirname $0`; pwd)" +DISTDIR="$FWDIR/dist" + +# Get version from SBT +VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2) +echo "Making distribution for Spark $VERSION in $DISTDIR..." + +# Build fat JAR +$FWDIR/sbt/sbt "repl/assembly" + +# Make directories +rm -rf "$DISTDIR" +mkdir -p "$DISTDIR/jars" +echo "$VERSION" >$DISTDIR/RELEASE + +# Copy jars +cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" + +# Copy other things +cp -r "$FWDIR/bin" "$DISTDIR" +cp -r "$FWDIR/conf" "$DISTDIR" +cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" \ No newline at end of file From cee05a174897af294f52cbda551da09cb869557e Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 23:55:09 -0700 Subject: [PATCH 005/221] Copy restore-TTY functions from Scala script so binary distros don't need 'scala' installed --- spark-shell | 39 ++++++++++++++++++++++++++++++++++++--- 1 file changed, 36 insertions(+), 3 deletions(-) diff --git a/spark-shell b/spark-shell index 574ae2104d..afbb7a9a8e 100755 --- a/spark-shell +++ b/spark-shell @@ -1,4 +1,37 @@ -#!/bin/sh +#!/bin/bash --posix FWDIR="`dirname $0`" -export SPARK_LAUNCH_WITH_SCALA=1 -exec $FWDIR/run spark.repl.Main "$@" + +# 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 +saved_stty="" + +# restore stty settings (echo in particular) +function restoreSttySettings() { + stty $saved_stty + saved_stty="" +} + +function onExit() { + if [[ "$saved_stty" != "" ]]; then + restoreSttySettings + fi + exit $exit_status +} + +# to reenable echo if we are interrupted before completing. +trap onExit INT + +# save terminal settings +saved_stty=$(stty -g 2>/dev/null) +# clear on error so we don't later try to restore them +if [[ ! $? ]]; then + saved_stty="" +fi + +$FWDIR/run spark.repl.Main "$@" + +# record the exit status lest it be overwritten: +# then reenable echo and propagate the code. +exit_status=$? +onExit From c3d11d0d57739ec3f08783f71bf4d0efdec3d627 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 00:02:00 -0700 Subject: [PATCH 006/221] Get rid of debugging statements --- run | 3 --- 1 file changed, 3 deletions(-) diff --git a/run b/run index 30a2885a4d..646d12c1eb 100755 --- a/run +++ b/run @@ -64,8 +64,6 @@ export JAVA_OPTS # Check if this is a binary distribution or source distribution # and build up the classpath appropriately if [ -f "$FWDIR/RELEASE" ]; then - echo "This is a binary distribution" - if [ `command -v java` ]; then RUNNER="java" else @@ -81,7 +79,6 @@ if [ -f "$FWDIR/RELEASE" ]; then # The JVM doesn't read JAVA_OPTS by default so we need to pass it in EXTRA_ARGS="$JAVA_OPTS" else - echo "This is a source distribution" . "$FWDIR/set-dev-classpath.sh" fi From 243d71cb066d888f3d1a9c613859522438b69ba8 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 00:15:58 -0700 Subject: [PATCH 007/221] Add deploy/testing procedure --- make-distribution.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/make-distribution.sh b/make-distribution.sh index 34f6d4a36b..855475864d 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -4,6 +4,14 @@ # The distribution directory defaults to dist/ but can be overridden below. # The distribution contains fat (assembly) jars that include the Scala library, # so it is completely self contained. +# It does not contain source or *.class files. +# +# Recommended deploy/testing procedure (standalone mode): +# 1) Rsync / deploy the dist/ dir to one host +# 2) cd to deploy dir; ./bin/start-master.sh +# 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. +# 4) ./bin/start-slave.sh 1 <> +# 5) MASTER="spark://my-master-ip:7077" ./spark-shell # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" From 982a686ff858399067de960fa62cc80f60c6fa32 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 11:14:21 -0700 Subject: [PATCH 008/221] Add -m and -c options to spark-shell for convenience --- spark-shell | 29 ++++++++++++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/spark-shell b/spark-shell index afbb7a9a8e..ea67a3e6b8 100755 --- a/spark-shell +++ b/spark-shell @@ -1,6 +1,33 @@ #!/bin/bash --posix +# +# Shell script for starting the Spark Shell REPL +# Options: +# -m Set MASTER to spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT +# -c Set the number of cores for REPL to use +# FWDIR="`dirname $0`" +for o in "$@"; do + if [ "$1" = "-m" -o "$1" = "--master" ]; then + shift + if [ -e "$FWDIR/conf/spark-env.sh" ]; then + . "$FWDIR/conf/spark-env.sh" + fi + if [ -z "$MASTER" ]; then + MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" + fi + export MASTER + fi + + if [ "$1" = "-c" -o "$1" = "--cores" ]; then + shift + if [ -n "$1" ]; then + OPTIONS="-Dspark.cores.max=$1" + shift + fi + fi +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 @@ -29,7 +56,7 @@ if [[ ! $? ]]; then saved_stty="" fi -$FWDIR/run spark.repl.Main "$@" +$FWDIR/run $OPTIONS spark.repl.Main "$@" # record the exit status lest it be overwritten: # then reenable echo and propagate the code. From 8dd78f80b578751df164772a01772aad26540ac9 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 15:01:06 -0700 Subject: [PATCH 009/221] Manually merge spark/master commit d92d3f7938dec954ea31de232f50cafd4b644065 --- set-dev-classpath.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/set-dev-classpath.sh b/set-dev-classpath.sh index 4d09bd4416..d031c56baf 100644 --- a/set-dev-classpath.sh +++ b/set-dev-classpath.sh @@ -74,10 +74,15 @@ if [ -e "$FWDIR/lib_managed" ]; then CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" fi CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" + +# Add the shaded JAR for Maven builds if [ -e $REPL_BIN_DIR/target ]; then for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do CLASSPATH="$CLASSPATH:$jar" done + # The shaded JAR doesn't contain examples, so include those separately + EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + CLASSPATH+=":$EXAMPLES_JAR" fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do @@ -90,11 +95,12 @@ if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; # Use the JAR from the SBT build export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` fi -if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then +if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` fi + # Figure out whether to run our class with java or with the scala launcher. # In most cases, we'd prefer to execute our process with java because scala # creates a shell script as the parent of its Java process, which makes it From 8113c55df8f4b5f34140ddba5e58e132e3dc2d23 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Fri, 28 Jun 2013 13:46:21 -0700 Subject: [PATCH 010/221] [Feedback] Get rid of -m, set MASTER from SPARK_MASTER_IP/PORT automagically --- spark-shell | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/spark-shell b/spark-shell index ea67a3e6b8..a8e72143fb 100755 --- a/spark-shell +++ b/spark-shell @@ -1,24 +1,14 @@ #!/bin/bash --posix # # 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. # Options: -# -m Set MASTER to spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT # -c Set the number of cores for REPL to use # FWDIR="`dirname $0`" for o in "$@"; do - if [ "$1" = "-m" -o "$1" = "--master" ]; then - shift - if [ -e "$FWDIR/conf/spark-env.sh" ]; then - . "$FWDIR/conf/spark-env.sh" - fi - if [ -z "$MASTER" ]; then - MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - fi - export MASTER - fi - if [ "$1" = "-c" -o "$1" = "--cores" ]; then shift if [ -n "$1" ]; then @@ -28,6 +18,17 @@ for o in "$@"; do fi done +# Set MASTER from spark-env if possible +if [ -z "$MASTER" ]; then + if [ -e "$FWDIR/conf/spark-env.sh" ]; then + . "$FWDIR/conf/spark-env.sh" + fi + if [[ "x" != "x$SPARK_MASTER_IP" && "y" != "y$SPARK_MASTER_PORT" ]]; then + MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" + export MASTER + fi +fi + # 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 From a948f0672541bd68be020f07134659aee2f38403 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 02:30:45 -0700 Subject: [PATCH 011/221] Suppress log messages in sbt test with two changes: 1. Set akka log level to ERROR before shutting down the actorSystem. This avoids akka log messages (like Spray) from falling back to INFO on the Stdout logger 2. Initialize netty to use SLF4J in LocalSparkContext. This ensures that stack trace thrown during shutdown is handled by SLF4J instead of stdout --- core/src/main/scala/spark/SparkEnv.scala | 2 ++ .../main/scala/spark/deploy/LocalSparkCluster.scala | 4 ++++ core/src/test/scala/spark/FileServerSuite.scala | 1 - core/src/test/scala/spark/LocalSparkContext.scala | 11 ++++++++++- 4 files changed, 16 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index ec59b4f48f..16b00d15aa 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -5,6 +5,7 @@ import serializer.Serializer import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider +import akka.event.{Logging => AkkaLogging} import spark.broadcast.BroadcastManager import spark.storage.BlockManager @@ -51,6 +52,7 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() + actorSystem.eventStream.setLogLevel(AkkaLogging.ErrorLevel) actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 55bb61b0cc..cb85419ae4 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -1,6 +1,7 @@ package spark.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.event.{Logging => AkkaLogging} import spark.deploy.worker.Worker import spark.deploy.master.Master @@ -43,8 +44,11 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I def stop() { logInfo("Shutting down local Spark cluster.") // Stop the workers before the master so they don't get upset that it disconnected + workerActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) workerActorSystems.foreach(_.shutdown()) workerActorSystems.foreach(_.awaitTermination()) + + masterActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) masterActorSystems.foreach(_.shutdown()) masterActorSystems.foreach(_.awaitTermination()) } diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index f1a35bced3..9c24ca430d 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -85,7 +85,6 @@ class FileServerSuite extends FunSuite with LocalSparkContext { in.close() _ * fileVal + _ * fileVal }.collect - println(result) assert(result.toSet === Set((1,200), (2,300), (3,500))) } diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala index 76d5258b02..bd184222ed 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -2,12 +2,21 @@ package spark import org.scalatest.Suite import org.scalatest.BeforeAndAfterEach +import org.scalatest.BeforeAndAfterAll + +import org.jboss.netty.logging.InternalLoggerFactory +import org.jboss.netty.logging.Slf4JLoggerFactory /** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */ -trait LocalSparkContext extends BeforeAndAfterEach { self: Suite => +trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ + override def beforeAll() { + InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory()); + super.beforeAll() + } + override def afterEach() { resetSparkContext() super.afterEach() From 7d6d9e6ab226579518f1c7fbe108c4e66acc6ed0 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 02:45:55 -0700 Subject: [PATCH 012/221] Set DriverSuite log level to WARN --- core/src/test/scala/spark/DriverSuite.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index 5e84b3a66a..31c3dd75fb 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -2,6 +2,9 @@ package spark import java.io.File +import org.apache.log4j.Logger +import org.apache.log4j.Level + import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.prop.TableDrivenPropertyChecks._ @@ -27,6 +30,7 @@ class DriverSuite extends FunSuite with Timeouts { */ object DriverWithoutCleanup { def main(args: Array[String]) { + Logger.getRootLogger().setLevel(Level.WARN) val sc = new SparkContext(args(0), "DriverWithoutCleanup") sc.parallelize(1 to 100, 4).count() } From 3350ad0d7fc3ecece78f87d7aa6a727e48b21c8c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 03:32:26 -0700 Subject: [PATCH 013/221] Catch RejectedExecution exception in Checkpoint handler. --- .../src/main/scala/spark/streaming/Checkpoint.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 66e67cbfa1..450e48d66e 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -8,7 +8,7 @@ import org.apache.hadoop.conf.Configuration import java.io._ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import java.util.concurrent.Executors - +import java.util.concurrent.RejectedExecutionException private[streaming] class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) @@ -91,7 +91,12 @@ class CheckpointWriter(checkpointDir: String) extends Logging { oos.writeObject(checkpoint) oos.close() bos.close() - executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + try { + executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + } catch { + case rej: RejectedExecutionException => + logError("Could not submit checkpoint task to the thread pool executor", rej) + } } def stop() { From d362d0f4117268bdef265041ff291700ddd49b43 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 03:33:32 -0700 Subject: [PATCH 014/221] Ignore stderr when calling cat on a non-existing file --- core/src/test/scala/spark/PipedRDDSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index 1c9ca50811..d263bb00e9 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -67,7 +67,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { test("pipe with non-zero exit status") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe("cat nonexistent_file") + val piped = nums.pipe(Seq("cat nonexistent_file", "2>", "/dev/null")) intercept[SparkException] { piped.collect() } From 4af0d63cb14db902cbd1dbdeeb68f1fcec4b2e97 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 10:42:43 -0700 Subject: [PATCH 015/221] Remove akka LogLevel fix as we no longer use spray --- core/src/main/scala/spark/SparkEnv.scala | 2 -- core/src/main/scala/spark/deploy/LocalSparkCluster.scala | 3 --- 2 files changed, 5 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 16b00d15aa..ec59b4f48f 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -5,7 +5,6 @@ import serializer.Serializer import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider -import akka.event.{Logging => AkkaLogging} import spark.broadcast.BroadcastManager import spark.storage.BlockManager @@ -52,7 +51,6 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() - actorSystem.eventStream.setLogLevel(AkkaLogging.ErrorLevel) actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index cb85419ae4..939f26b6f4 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -1,7 +1,6 @@ package spark.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import akka.event.{Logging => AkkaLogging} import spark.deploy.worker.Worker import spark.deploy.master.Master @@ -44,11 +43,9 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I def stop() { logInfo("Shutting down local Spark cluster.") // Stop the workers before the master so they don't get upset that it disconnected - workerActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) workerActorSystems.foreach(_.shutdown()) workerActorSystems.foreach(_.awaitTermination()) - masterActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) masterActorSystems.foreach(_.shutdown()) masterActorSystems.foreach(_.awaitTermination()) } From 8c1d1c98e0cea8d2b20ac10b84aa76d8e22a1661 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Mon, 8 Jul 2013 12:25:46 -0700 Subject: [PATCH 016/221] Explicitly set class loader for MesosSchedulerDriver callbacks. --- .../scheduler/mesos/MesosSchedulerBackend.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index ca7fab4cc5..e73b780fcb 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -43,8 +43,12 @@ private[spark] class MesosSchedulerBackend( // An ExecutorInfo for our tasks var execArgs: Array[Byte] = null + var classLoader: ClassLoader = null + override def start() { synchronized { + classLoader = Thread.currentThread.getContextClassLoader + new Thread("MesosSchedulerBackend driver") { setDaemon(true) override def run() { @@ -114,9 +118,16 @@ private[spark] class MesosSchedulerBackend( return execArgs } + private def setClassLoader() { + // Since native code starts the thread our callbacks run in, it may not correctly + // inherit and custom class loaders. Therefore, set the class loader manually. + Thread.currentThread.setContextClassLoader(classLoader) + } + override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { + setClassLoader() logInfo("Registered as framework ID " + frameworkId.getValue) registeredLock.synchronized { isRegistered = true @@ -142,6 +153,7 @@ private[spark] class MesosSchedulerBackend( * tasks are balanced across the cluster. */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { + setClassLoader() synchronized { // Build a big list of the offerable workers, and remember their indices so that we can // figure out which Offer to reply to for each worker @@ -224,6 +236,7 @@ private[spark] class MesosSchedulerBackend( } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + setClassLoader() val tid = status.getTaskId.getValue.toLong val state = TaskState.fromMesos(status.getState) synchronized { From bf4c9a5e0fca2dfc960120a7f3c5fab0b87e3850 Mon Sep 17 00:00:00 2001 From: Ameet Talwalkar Date: Mon, 8 Jul 2013 14:37:42 -0700 Subject: [PATCH 017/221] renamed with labeled prefix --- .../scala/spark/mllib/regression/LogisticRegression.scala | 2 +- .../spark/mllib/regression/LogisticRegressionGenerator.scala | 2 +- .../main/scala/spark/mllib/regression/RidgeRegression.scala | 2 +- .../spark/mllib/regression/RidgeRegressionGenerator.scala | 2 +- mllib/src/main/scala/spark/mllib/util/MLUtils.scala | 4 ++-- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index 448ab9dce9..e4db7bb9b7 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -150,7 +150,7 @@ object LogisticRegression { System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") - val data = MLUtils.loadData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)) val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) sc.stop() diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala index 9f6abab70b..6e7c023bac 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala @@ -35,7 +35,7 @@ object LogisticRegressionGenerator { (y, x) } - MLUtils.saveData(data, outputPath) + MLUtils.saveLabeledData(data, outputPath) sc.stop() } } diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index f66025bc0b..5f813df402 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -187,7 +187,7 @@ object RidgeRegression { System.exit(1) } val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)) val model = RidgeRegression.train(data, 0, 1000) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala index c9ac4a8b07..b83f505d8e 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala @@ -49,7 +49,7 @@ object RidgeRegressionGenerator { } } - MLUtils.saveData(data, outputPath) + MLUtils.saveLabeledData(data, outputPath) sc.stop() } } diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala index 0a4a037c71..08a031dded 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -19,7 +19,7 @@ object MLUtils { * @return An RDD of tuples. For each tuple, the first element is the label, and the second * element represents the feature values (an array of Double). */ - def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { + def loadLabeledData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { sc.textFile(dir).map { line => val parts = line.split(",") val label = parts(0).toDouble @@ -28,7 +28,7 @@ object MLUtils { } } - def saveData(data: RDD[(Double, Array[Double])], dir: String) { + def saveLabeledData(data: RDD[(Double, Array[Double])], dir: String) { val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) dataStr.saveAsTextFile(dir) } From afdaf430bd8d7da3ee3323cf03b00d4214159626 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Jul 2013 15:38:29 -0700 Subject: [PATCH 018/221] Explicit dependencies for scala-library and scalap to prevent 2.9.2 vs. 2.9.3 problems --- core/pom.xml | 10 ++++++++++ examples/pom.xml | 5 +++++ streaming/pom.xml | 5 +++++ 3 files changed, 20 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 5edafb3706..39dba46d7a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -55,6 +55,16 @@ com.typesafe.akka akka-slf4j + + org.scala-lang + scalap + 2.9.3 + + + org.scala-lang + scala-library + 2.9.3 + net.liftweb lift-json_2.9.2 diff --git a/examples/pom.xml b/examples/pom.xml index 78ec58729b..7d9769e8e4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -15,6 +15,11 @@ http://spark-project.org/ + + org.scala-lang + scala-library + 2.9.3 + org.eclipse.jetty jetty-server diff --git a/streaming/pom.xml b/streaming/pom.xml index 4dc9a19d51..07725f9484 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -58,6 +58,11 @@ twitter4j-stream 3.0.3 + + org.scala-lang + scala-library + 2.9.3 + com.typesafe.akka akka-zeromq From 0b39d66f3f815f2d406d83a41db0f46d097baaa7 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Jul 2013 16:07:09 -0700 Subject: [PATCH 019/221] pom cleanup --- core/pom.xml | 2 -- examples/pom.xml | 1 - pom.xml | 10 ++++++++++ streaming/pom.xml | 1 - 4 files changed, 10 insertions(+), 4 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 39dba46d7a..dbb2da9a9c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -58,12 +58,10 @@ org.scala-lang scalap - 2.9.3 org.scala-lang scala-library - 2.9.3 net.liftweb diff --git a/examples/pom.xml b/examples/pom.xml index 7d9769e8e4..1976765c3d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -18,7 +18,6 @@ org.scala-lang scala-library - 2.9.3 org.eclipse.jetty diff --git a/pom.xml b/pom.xml index 017c242464..63dd80b5fe 100644 --- a/pom.xml +++ b/pom.xml @@ -241,6 +241,16 @@ jline ${scala.version} + + org.scala-lang + scala-library + ${scala.version} + + + org.scala-lang + scalap + ${scala.version} + log4j diff --git a/streaming/pom.xml b/streaming/pom.xml index 07725f9484..2fb5bbdeb5 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -61,7 +61,6 @@ org.scala-lang scala-library - 2.9.3 com.typesafe.akka From c1d44be80580f0fad6bb1805bbcf74a34f536d8c Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Tue, 9 Jul 2013 15:18:28 +0800 Subject: [PATCH 020/221] Bug fix: SPARK-796 --- .../spark/deploy/yarn/ApplicationMaster.scala | 49 ++++++++++++------- 1 file changed, 32 insertions(+), 17 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index f19648ec68..9bc692d480 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -27,6 +27,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var yarnAllocator: YarnAllocationHandler = null + private var isFinished:Boolean = false def run() { @@ -68,10 +69,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Wait for the user class to Finish userThread.join() - - // Finish the ApplicationMaster - finishApplicationMaster() - // TODO: Exit based on success/failure + System.exit(0) } @@ -131,10 +129,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e .getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { - // Copy - var mainArgs: Array[String] = new Array[String](args.userArgs.size()) - args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) - mainMethod.invoke(null, mainArgs) + try{ + // Copy + var mainArgs: Array[String] = new Array[String](args.userArgs.size()) + args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) + mainMethod.invoke(null, mainArgs) + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + } catch { + case th: Throwable => + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) + logError("Finish ApplicationMaster with ",th) + } } } t.start() @@ -235,14 +240,22 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } */ - - def finishApplicationMaster() { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - // TODO: Check if the application has failed or succeeded - finishReq.setFinishApplicationStatus(FinalApplicationStatus.SUCCEEDED) - resourceManager.finishApplicationMaster(finishReq) + + def finishApplicationMaster(status: FinalApplicationStatus) { + + synchronized { + if(isFinished){ + return + } + isFinished = true + + logInfo("finishApplicationMaster with "+status) + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + resourceManager.finishApplicationMaster(finishReq) + } } } @@ -291,7 +304,9 @@ object ApplicationMaster { logInfo("Invoking sc stop from shutdown hook") sc.stop() // best case ... - for (master <- applicationMasters) master.finishApplicationMaster + for (master <- applicationMasters) { + master.finishApplicationMaster(FinalApplicationStatus.KILLED) + } } } ) } From e47253e0cca1359b49f113dbf258c2c204e3bfc1 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 9 Jul 2013 01:13:12 -0700 Subject: [PATCH 021/221] Reset ClassLoader in MesosSchedulerBackend, too. (per review comments). Also set ClassLoader for all mesos callbacks, not just statusUpdate, registered. --- .../mesos/MesosSchedulerBackend.scala | 148 ++++++++++-------- 1 file changed, 87 insertions(+), 61 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index e73b780fcb..e83368b98d 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -118,20 +118,28 @@ private[spark] class MesosSchedulerBackend( return execArgs } - private def setClassLoader() { - // Since native code starts the thread our callbacks run in, it may not correctly - // inherit and custom class loaders. Therefore, set the class loader manually. + private def setClassLoader(): ClassLoader = { + val oldClassLoader = Thread.currentThread.getContextClassLoader Thread.currentThread.setContextClassLoader(classLoader) + return oldClassLoader + } + + private def restoreClassLoader(oldClassLoader: ClassLoader) { + Thread.currentThread.setContextClassLoader(oldClassLoader) } override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - setClassLoader() - logInfo("Registered as framework ID " + frameworkId.getValue) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() + val oldClassLoader = setClassLoader() + try { + logInfo("Registered as framework ID " + frameworkId.getValue) + registeredLock.synchronized { + isRegistered = true + registeredLock.notifyAll() + } + } finally { + restoreClassLoader(oldClassLoader) } } @@ -153,50 +161,54 @@ private[spark] class MesosSchedulerBackend( * tasks are balanced across the cluster. */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { - setClassLoader() - synchronized { - // Build a big list of the offerable workers, and remember their indices so that we can - // figure out which Offer to reply to for each worker - val offerableIndices = new ArrayBuffer[Int] - val offerableWorkers = new ArrayBuffer[WorkerOffer] + val oldClassLoader = setClassLoader() + try { + synchronized { + // Build a big list of the offerable workers, and remember their indices so that we can + // figure out which Offer to reply to for each worker + val offerableIndices = new ArrayBuffer[Int] + val offerableWorkers = new ArrayBuffer[WorkerOffer] - def enoughMemory(o: Offer) = { - val mem = getResource(o.getResourcesList, "mem") - val slaveId = o.getSlaveId.getValue - mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId) - } + def enoughMemory(o: Offer) = { + val mem = getResource(o.getResourcesList, "mem") + val slaveId = o.getSlaveId.getValue + mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId) + } - for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { - offerableIndices += index - offerableWorkers += new WorkerOffer( - offer.getSlaveId.getValue, - offer.getHostname, - getResource(offer.getResourcesList, "cpus").toInt) - } + for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { + offerableIndices += index + offerableWorkers += new WorkerOffer( + offer.getSlaveId.getValue, + offer.getHostname, + getResource(offer.getResourcesList, "cpus").toInt) + } - // Call into the ClusterScheduler - val taskLists = scheduler.resourceOffers(offerableWorkers) + // Call into the ClusterScheduler + val taskLists = scheduler.resourceOffers(offerableWorkers) - // Build a list of Mesos tasks for each slave - val mesosTasks = offers.map(o => Collections.emptyList[MesosTaskInfo]()) - for ((taskList, index) <- taskLists.zipWithIndex) { - if (!taskList.isEmpty) { - val offerNum = offerableIndices(index) - val slaveId = offers(offerNum).getSlaveId.getValue - slaveIdsWithExecutors += slaveId - mesosTasks(offerNum) = new JArrayList[MesosTaskInfo](taskList.size) - for (taskDesc <- taskList) { - taskIdToSlaveId(taskDesc.taskId) = slaveId - mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId)) + // Build a list of Mesos tasks for each slave + val mesosTasks = offers.map(o => Collections.emptyList[MesosTaskInfo]()) + for ((taskList, index) <- taskLists.zipWithIndex) { + if (!taskList.isEmpty) { + val offerNum = offerableIndices(index) + val slaveId = offers(offerNum).getSlaveId.getValue + slaveIdsWithExecutors += slaveId + mesosTasks(offerNum) = new JArrayList[MesosTaskInfo](taskList.size) + for (taskDesc <- taskList) { + taskIdToSlaveId(taskDesc.taskId) = slaveId + mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId)) + } } } - } - // Reply to the offers - val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? - for (i <- 0 until offers.size) { - d.launchTasks(offers(i).getId, mesosTasks(i), filters) + // Reply to the offers + val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? + for (i <- 0 until offers.size) { + d.launchTasks(offers(i).getId, mesosTasks(i), filters) + } } + } finally { + restoreClassLoader(oldClassLoader) } } @@ -236,24 +248,33 @@ private[spark] class MesosSchedulerBackend( } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { - setClassLoader() - val tid = status.getTaskId.getValue.toLong - val state = TaskState.fromMesos(status.getState) - synchronized { - if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { - // We lost the executor on this slave, so remember that it's gone - slaveIdsWithExecutors -= taskIdToSlaveId(tid) - } - if (isFinished(status.getState)) { - taskIdToSlaveId.remove(tid) + val oldClassLoader = setClassLoader() + try { + val tid = status.getTaskId.getValue.toLong + val state = TaskState.fromMesos(status.getState) + synchronized { + if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { + // We lost the executor on this slave, so remember that it's gone + slaveIdsWithExecutors -= taskIdToSlaveId(tid) + } + if (isFinished(status.getState)) { + taskIdToSlaveId.remove(tid) + } } + scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) + } finally { + restoreClassLoader(oldClassLoader) } - scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) } override def error(d: SchedulerDriver, message: String) { - logError("Mesos error: " + message) - scheduler.error(message) + val oldClassLoader = setClassLoader() + try { + logError("Mesos error: " + message) + scheduler.error(message) + } finally { + restoreClassLoader(oldClassLoader) + } } override def stop() { @@ -269,11 +290,16 @@ private[spark] class MesosSchedulerBackend( override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { - logInfo("Mesos slave lost: " + slaveId.getValue) - synchronized { - slaveIdsWithExecutors -= slaveId.getValue + val oldClassLoader = setClassLoader() + try { + logInfo("Mesos slave lost: " + slaveId.getValue) + synchronized { + slaveIdsWithExecutors -= slaveId.getValue + } + scheduler.executorLost(slaveId.getValue, reason) + } finally { + restoreClassLoader(oldClassLoader) } - scheduler.executorLost(slaveId.getValue, reason) } override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { From aaa7b081df760a29ce5cdcd51d6b71422cba68d5 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Tue, 9 Jul 2013 20:03:01 +0800 Subject: [PATCH 022/221] according to mridulm's comments to adjust the code --- .../spark/deploy/yarn/ApplicationMaster.scala | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 9bc692d480..776db201f9 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -122,23 +122,26 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } } - + private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader) .getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { + var successed = false try{ // Copy var mainArgs: Array[String] = new Array[String](args.userArgs.size()) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) mainMethod.invoke(null, mainArgs) - ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - } catch { - case th: Throwable => + successed = true + } finally { + if(successed){ + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + }else{ ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) - logError("Finish ApplicationMaster with ",th) + } } } } @@ -248,14 +251,15 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e return } isFinished = true - - logInfo("finishApplicationMaster with "+status) - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - resourceManager.finishApplicationMaster(finishReq) } + + logInfo("finishApplicationMaster with " + status) + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + resourceManager.finishApplicationMaster(finishReq) + } } @@ -304,6 +308,9 @@ object ApplicationMaster { logInfo("Invoking sc stop from shutdown hook") sc.stop() // best case ... + // due to the sparkContext is stopped and ApplicationMaster is down, + // the status of registered masters should be set KILLED better than FAILED. + // need discussion for (master <- applicationMasters) { master.finishApplicationMaster(FinalApplicationStatus.KILLED) } From 13fc6f248c7231501d8c53ad13641fa996e06be4 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 9 Jul 2013 13:42:51 -0700 Subject: [PATCH 023/221] Clean commit of log paging --- .../deploy/master/ui/ApplicationPage.scala | 8 ++- .../spark/deploy/worker/ui/IndexPage.scala | 8 ++- .../spark/deploy/worker/ui/WorkerWebUI.scala | 61 +++++++++++++++++++ 3 files changed, 75 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 33a16b5d84..ea88421532 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -38,7 +38,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { state.completedApps.find(_.id == appId).getOrElse(null) }) - val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs", "Log Pages") val executors = app.executors.values.toSeq val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) @@ -95,6 +95,12 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { stderr + + stdout + stderr + } } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index e466129c1a..7cf98b473e 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -29,7 +29,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs", "Log Pages") val runningExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) val finishedExecutorTable = @@ -93,6 +93,12 @@ private[spark] class IndexPage(parent: WorkerWebUI) { stderr + + stdout-page + stderr-page + } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 16564d5619..5b0c785b00 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -13,6 +13,10 @@ import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ +import scala.xml._ +import spark.ui.UIUtils +import scala.io.Source._ + /** * Web UI server for the standalone worker. */ @@ -33,6 +37,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val handlers = Array[(String, Handler)]( ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), ("/log", (request: HttpServletRequest) => log(request)), + ("/logPage", (request: HttpServletRequest) => logPage(request)), ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), ("*", (request: HttpServletRequest) => indexPage.render(request)) ) @@ -65,6 +70,62 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option pre + Utils.lastNBytes(path, math.min(numBytes, maxBytes)) } + def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { + val appId = request.getParameter("appId") + val executorId = request.getParameter("executorId") + val logType = request.getParameter("logType") + val getOffset = request.getParameter("offset") + val getLineLength = request.getParameter("lineLength") + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) + val source = fromFile(path) + val lines = source.getLines().toArray + val logLength = lines.length + val offset = { + if (getOffset == null) 0 + else if (getOffset.toInt < 0) 0 + else getOffset.toInt + } + val lineLength = { + if (getLineLength == null) 0 + else getLineLength.toInt + } + val logText = "" + lines.slice(offset, offset+lineLength).mkString("\n") + "" + val logXML = XML.loadString(logText) + val backButton = + if (offset > 0) { + if (offset-lineLength < 0) { + + } + else { + + } + } + else { + + } + val nextButton = + if (offset+lineLength < logLength) { + + } + else { + + } + val content = + + +
+ {backButton} + {nextButton} +

+
{logXML}
+ {backButton} + {nextButton} + + + source.close() + UIUtils.basicSparkPage(content, "Log Page for " + appId) + } + def stop() { server.foreach(_.stop()) } From b6072b58bf795093df0e5f8424413fcd0cce6323 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 9 Jul 2013 17:25:10 -0700 Subject: [PATCH 024/221] Fixes style, makes "std__-page" consistent, reads only parts of files --- core/src/main/scala/spark/Utils.scala | 14 +++++ .../deploy/master/ui/ApplicationPage.scala | 8 +-- .../spark/deploy/worker/ui/IndexPage.scala | 4 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 54 +++++++++---------- 4 files changed, 45 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 64547bbdcd..dfc30469cf 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -635,6 +635,20 @@ private object Utils extends Logging { Source.fromBytes(buff).mkString } + /** Return an array containing part of a file from byte 'a' to 'b'. */ + def offsetBytes(path: String, a: Int, b: Int): String = { + val file = new File(path) + val length = file.length() + val buff = new Array[Byte](math.min(b-a, length.toInt)) + val skip = math.max(0, a) + val stream = new FileInputStream(file) + + stream.skip(skip) + stream.read(buff) + stream.close() + Source.fromBytes(buff).mkString + } + /** * Clone an object using a Spark serializer. */ diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index ea88421532..cc32728c1c 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -96,10 +96,10 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stderr - stdout - stderr + stdout-page + stderr-page } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 7cf98b473e..d532aa9e95 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -94,9 +94,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { .format(executor.appId, executor.execId)}>stderr - stdout-page - stderr-page diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 5b0c785b00..b018f80c85 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -9,13 +9,13 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} +import scala.io.Source._ +import scala.xml._ + import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ - -import scala.xml._ import spark.ui.UIUtils -import scala.io.Source._ /** * Web UI server for the standalone worker. @@ -74,42 +74,39 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val getOffset = request.getParameter("offset") - val getLineLength = request.getParameter("lineLength") + val offset = Option(request.getParameter("offset")).map(_.toInt).getOrElse(0) + + val maxBytes = 1024 * 1024 + val defaultBytes = 100 * 1024 + val byteLength = Option(request.getParameter("byteLength")).flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val source = fromFile(path) - val lines = source.getLines().toArray - val logLength = lines.length - val offset = { - if (getOffset == null) 0 - else if (getOffset.toInt < 0) 0 - else getOffset.toInt - } - val lineLength = { - if (getLineLength == null) 0 - else getLineLength.toInt - } - val logText = "" + lines.slice(offset, offset+lineLength).mkString("\n") + "" - val logXML = XML.loadString(logText) + val logLength = new File(path).length() + val logPageLength = math.min(byteLength, maxBytes) + val logText = {Utils.offsetBytes(path, offset, offset+logPageLength)} + val backButton = if (offset > 0) { - if (offset-lineLength < 0) { - - } - else { - - } + + + } else { } + val nextButton = - if (offset+lineLength < logLength) { - + if (offset+logPageLength < logLength) { + + + } else { } + val content = @@ -117,12 +114,11 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option {backButton} {nextButton}

-
{logXML}
+
{logText}
{backButton} {nextButton} - source.close() UIUtils.basicSparkPage(content, "Log Page for " + appId) } From ce18b50d5ff37dc5c558d0602321a61887dd8b48 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Wed, 10 Jul 2013 19:11:43 +0800 Subject: [PATCH 025/221] set SUCCEEDED for all master in shutdown hook --- .../scala/spark/deploy/yarn/ApplicationMaster.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 776db201f9..68bb36d316 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -135,6 +135,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e var mainArgs: Array[String] = new Array[String](args.userArgs.size()) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) mainMethod.invoke(null, mainArgs) + // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR + // userThread will stop here unless it has uncaught exception thrown out + // It need shutdown hook to set SUCCEEDED successed = true } finally { if(successed){ @@ -308,11 +311,8 @@ object ApplicationMaster { logInfo("Invoking sc stop from shutdown hook") sc.stop() // best case ... - // due to the sparkContext is stopped and ApplicationMaster is down, - // the status of registered masters should be set KILLED better than FAILED. - // need discussion for (master <- applicationMasters) { - master.finishApplicationMaster(FinalApplicationStatus.KILLED) + master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } } } ) From 620a6974c6603f1c0e5a7cea8f0387a5d18f2e5e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 10:20:53 -0700 Subject: [PATCH 026/221] Allows for larger files, refactors lastNBytes, removes old Log column, fixes imports, uses map --- core/src/main/scala/spark/Utils.scala | 21 +++++++------------ .../deploy/master/ui/ApplicationPage.scala | 12 +++-------- .../spark/deploy/worker/ui/IndexPage.scala | 12 +++-------- .../spark/deploy/worker/ui/WorkerWebUI.scala | 7 ++----- 4 files changed, 15 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index dfc30469cf..849edc13ee 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,12 +621,12 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } - /** Return a string containing the last `n` bytes of a file. */ - def lastNBytes(path: String, n: Int): String = { + /** Return a string containing part of a file from byte 'a' to 'b'. */ + def offsetBytes(path: String, a: Long, b: Long): String = { val file = new File(path) val length = file.length() - val buff = new Array[Byte](math.min(n, length.toInt)) - val skip = math.max(0, length - n) + val buff = new Array[Byte](math.min((b-a).toInt, length.toInt)) + val skip = math.max(0, a) val stream = new FileInputStream(file) stream.skip(skip) @@ -635,18 +635,11 @@ private object Utils extends Logging { Source.fromBytes(buff).mkString } - /** Return an array containing part of a file from byte 'a' to 'b'. */ - def offsetBytes(path: String, a: Int, b: Int): String = { + /** Return a string containing the last `n` bytes of a file. */ + def lastNBytes(path: String, n: Int): String = { val file = new File(path) val length = file.length() - val buff = new Array[Byte](math.min(b-a, length.toInt)) - val skip = math.max(0, a) - val stream = new FileInputStream(file) - - stream.skip(skip) - stream.read(buff) - stream.close() - Source.fromBytes(buff).mkString + offsetBytes(path, length-n, length) } /** diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index cc32728c1c..49ced0d320 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -38,7 +38,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { state.completedApps.find(_.id == appId).getOrElse(null) }) - val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Log Pages") val executors = app.executors.values.toSeq val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) @@ -89,17 +89,11 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.cores} {executor.memory} {executor.state} - - stdout - stderr - stdout-page + .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stdout stderr-page + .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stderr } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index d532aa9e95..af9943853f 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -29,7 +29,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Log Pages") val runningExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) val finishedExecutorTable = @@ -87,17 +87,11 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
  • User: {executor.appDesc.user}
  • - - stdout - stderr - stdout-page + .format(executor.appId, executor.execId)}>stdout stderr-page + .format(executor.appId, executor.execId)}>stderr } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index b018f80c85..602881d5f1 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -9,9 +9,6 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import scala.io.Source._ -import scala.xml._ - import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ @@ -74,11 +71,11 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toInt).getOrElse(0) + val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0).asInstanceOf[Long] val maxBytes = 1024 * 1024 val defaultBytes = 100 * 1024 - val byteLength = Option(request.getParameter("byteLength")).flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val logLength = new File(path).length() From 24705d0f46ce536bf829660f4506dcffd9ff799a Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 10 Jul 2013 10:33:11 -0700 Subject: [PATCH 027/221] adding takeOrdered() to RDD --- core/src/main/scala/spark/RDD.scala | 12 ++++++++++++ core/src/test/scala/spark/RDDSuite.scala | 18 ++++++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 106fb2960f..af52040fa6 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -781,6 +781,18 @@ abstract class RDD[T: ClassManifest]( }.toArray } + /** + * Returns the top K elements from this RDD as defined by + * the specified implicit Ordering[T] and maintains the + * ordering. + * @param num the number of top elements to return + * @param ord the implicit ordering for T + * @return an array of top elements + */ + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { + top(num)(ord.reverse).sorted(ord) + } + /** * Save this RDD as a text file, using string representations of elements. */ diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index e41ae385c0..fe17d1d5e7 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -252,6 +252,24 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(topK.sorted === Array("b", "a")) } + test("takeOrdered with predefined ordering") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + val rdd = sc.makeRDD(nums, 2) + val sortedTopK = rdd.takeOrdered(5) + assert(sortedTopK.size === 5) + assert(sortedTopK === Array(1, 2, 3, 4, 5)) + } + + test("takeOrdered with custom ordering") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + implicit val ord = implicitly[Ordering[Int]].reverse + val rdd = sc.makeRDD(nums, 2) + val sortedTopK = rdd.takeOrdered(5) + assert(sortedTopK.size === 5) + assert(sortedTopK === Array(10, 9, 8, 7, 6)) + assert(sortedTopK === nums.sorted(ord).take(5)) + } + test("takeSample") { val data = sc.parallelize(1 to 100, 2) for (seed <- 1 to 5) { From ee4ce2fc51112387f28d7b422969ca2e9736e95f Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 10 Jul 2013 10:46:04 -0700 Subject: [PATCH 028/221] adding takeOrdered to java API --- .../scala/spark/api/java/JavaRDDLike.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index b555f2030a..94b95af714 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -384,4 +384,29 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]] top(num, comp) } + + /** + * Returns the top K elements from this RDD as defined by + * the specified Comparator[T] and maintains the order. + * @param num the number of top elements to return + * @param comp the comparator that defines the order + * @return an array of top elements + */ + def takeOrdered(num: Int, comp: Comparator[T]): JList[T] = { + import scala.collection.JavaConversions._ + val topElems = rdd.takeOrdered(num)(Ordering.comparatorToOrdering(comp)) + val arr: java.util.Collection[T] = topElems.toSeq + new java.util.ArrayList(arr) + } + + /** + * Returns the top K elements from this RDD using the + * natural ordering for T while maintain the order. + * @param num the number of top elements to return + * @return an array of top elements + */ + def takeOrdered(num: Int): JList[T] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]] + takeOrdered(num, comp) + } } From cfb6447ac4903a870dd268836dc0d8952491d591 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 11:47:57 -0700 Subject: [PATCH 029/221] Fixed for nonexistent bytes, added unit tests, changed stdout-page to stdout --- core/src/main/scala/spark/Utils.scala | 6 ++-- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 18 ++++++++---- core/src/test/scala/spark/ui/UISuite.scala | 28 +++++++++++++++++++ 5 files changed, 46 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 849edc13ee..512ac92d89 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -625,8 +625,10 @@ private object Utils extends Logging { def offsetBytes(path: String, a: Long, b: Long): String = { val file = new File(path) val length = file.length() - val buff = new Array[Byte](math.min((b-a).toInt, length.toInt)) - val skip = math.max(0, a) + val B = math.min(length, b) + val A = math.max(0, a) + val buff = new Array[Byte]((B-A).toInt) + val skip = A val stream = new FileInputStream(file) stream.skip(skip) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 49ced0d320..dae9995779 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -38,7 +38,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { state.completedApps.find(_.id == appId).getOrElse(null) }) - val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") val executors = app.executors.values.toSeq val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index af9943853f..a9c6c6d519 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -29,7 +29,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") val runningExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) val finishedExecutorTable = diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 602881d5f1..24356b0f63 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -71,7 +71,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0).asInstanceOf[Long] + val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0L) val maxBytes = 1024 * 1024 val defaultBytes = 100 * 1024 @@ -80,12 +80,18 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val logLength = new File(path).length() val logPageLength = math.min(byteLength, maxBytes) - val logText = {Utils.offsetBytes(path, offset, offset+logPageLength)} + + val fixedOffset = + if (offset < 0) 0 + else if (offset > logLength) logLength + else offset + + val logText = {Utils.offsetBytes(path, fixedOffset, fixedOffset+logPageLength)} val backButton = - if (offset > 0) { + if (fixedOffset > 0) { + .format(appId, executorId, logType, math.max(fixedOffset-logPageLength, 0), logPageLength)}> } @@ -94,9 +100,9 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } val nextButton = - if (offset+logPageLength < logLength) { + if (fixedOffset+logPageLength < logLength) { + format(appId, executorId, logType, fixedOffset+logPageLength, logPageLength)}> } diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index e4bb3abc33..b7a822c4bc 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -74,4 +74,32 @@ class UISuite extends FunSuite { FileUtils.deleteDirectory(tmpDir) } + + test("reading offset bytes of a file") { + val tmpDir2 = Files.createTempDir() + val f1Path = tmpDir2 + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) + f1.close() + + // Read first few bytes + assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") + + // Read some middle bytes + assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") + + // Read last few bytes + assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") + + //Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") + + //Read some nonexistent bytes at the end + assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") + + //Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + + FileUtils.deleteDirectory(tmpDir2) + } } From 04263e4d4609355d8e0779dea472fbb1aff5ef4f Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:15:42 -0700 Subject: [PATCH 030/221] Made some minor style changes --- core/src/main/scala/spark/Utils.scala | 11 +++++------ core/src/test/scala/spark/ui/UISuite.scala | 6 +++--- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 512ac92d89..c5627c8419 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -622,16 +622,15 @@ private object Utils extends Logging { } /** Return a string containing part of a file from byte 'a' to 'b'. */ - def offsetBytes(path: String, a: Long, b: Long): String = { + def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) val length = file.length() - val B = math.min(length, b) - val A = math.max(0, a) - val buff = new Array[Byte]((B-A).toInt) - val skip = A + val effectiveStart = math.min(length, start) + val effectiveEnd = math.max(0, end) + val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) val stream = new FileInputStream(file) - stream.skip(skip) + stream.skip(effectiveStart) stream.read(buff) stream.close() Source.fromBytes(buff).mkString diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index b7a822c4bc..ab174732e3 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -91,13 +91,13 @@ class UISuite extends FunSuite { // Read last few bytes assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") - //Read some nonexistent bytes in the beginning + // Read some nonexistent bytes in the beginning assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") - //Read some nonexistent bytes at the end + // Read some nonexistent bytes at the end assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") - //Read some nonexistent bytes on both ends + // Read some nonexistent bytes on both ends assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") FileUtils.deleteDirectory(tmpDir2) From 0d4580360ba4bbcd2f73877743d746c071a92e34 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:24:26 -0700 Subject: [PATCH 031/221] Fixed docstring of offsetBytes to match params and wrapped for 100+ character lines --- core/src/main/scala/spark/Utils.scala | 2 +- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c5627c8419..1c5af2700a 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,7 +621,7 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } - /** Return a string containing part of a file from byte 'a' to 'b'. */ + /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) val length = file.length() diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 24356b0f63..1f4ba5de60 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -91,7 +91,8 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val backButton = if (fixedOffset > 0) { + .format(appId, executorId, logType, math.max(fixedOffset-logPageLength, 0), + logPageLength)}> } From 5f8a20b4a8a4cb7e4b14ee989f1b4f83981ce61a Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:53:39 -0700 Subject: [PATCH 032/221] Moved unit tests for Utils from UISuite to UtilsSuite --- core/src/test/scala/spark/UtilsSuite.scala | 71 ++++++++++++++++++++++ core/src/test/scala/spark/ui/UISuite.scala | 71 ---------------------- 2 files changed, 71 insertions(+), 71 deletions(-) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 4a113e16bf..73050a1d3b 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -71,5 +71,76 @@ class UtilsSuite extends FunSuite { assert(Utils.splitCommandString("''") === Seq("")) assert(Utils.splitCommandString("\"\"") === Seq("")) } + + test("string formatting of time durations") { + val second = 1000 + val minute = second * 60 + val hour = minute * 60 + def str = Utils.msDurationToString(_) + + assert(str(123) === "123 ms") + assert(str(second) === "1.0 s") + assert(str(second + 462) === "1.5 s") + assert(str(hour) === "1.00 h") + assert(str(minute) === "1.0 m") + assert(str(minute + 4 * second + 34) === "1.1 m") + assert(str(10 * hour + minute + 4 * second) === "10.02 h") + assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") + } + + test("reading last n bytes of a file") { + val tmpDir = Files.createTempDir() + + // File smaller than limit + val f1Path = tmpDir + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) + f1.close() + assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") + + // File larger than limit + val f2Path = tmpDir + "/f2" + val f2 = new FileOutputStream(f2Path) + f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f2.close() + assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") + + // Request limit too + val f3Path = tmpDir + "/f2" + val f3 = new FileOutputStream(f3Path) + f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f3.close() + assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") + + FileUtils.deleteDirectory(tmpDir) + } + + test("reading offset bytes of a file") { + val tmpDir2 = Files.createTempDir() + val f1Path = tmpDir2 + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) + f1.close() + + // Read first few bytes + assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") + + // Read some middle bytes + assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") + + // Read last few bytes + assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") + + // Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") + + // Read some nonexistent bytes at the end + assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") + + // Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + + FileUtils.deleteDirectory(tmpDir2) + } } diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index ab174732e3..ef2e9c2ec4 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -31,75 +31,4 @@ class UISuite extends FunSuite { case Failure (e) => } } - - test("string formatting of time durations") { - val second = 1000 - val minute = second * 60 - val hour = minute * 60 - def str = Utils.msDurationToString(_) - - assert(str(123) === "123 ms") - assert(str(second) === "1.0 s") - assert(str(second + 462) === "1.5 s") - assert(str(hour) === "1.00 h") - assert(str(minute) === "1.0 m") - assert(str(minute + 4 * second + 34) === "1.1 m") - assert(str(10 * hour + minute + 4 * second) === "10.02 h") - assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") - } - - test("reading last n bytes of a file") { - val tmpDir = Files.createTempDir() - - // File smaller than limit - val f1Path = tmpDir + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) - f1.close() - assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") - - // File larger than limit - val f2Path = tmpDir + "/f2" - val f2 = new FileOutputStream(f2Path) - f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f2.close() - assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") - - // Request limit too - val f3Path = tmpDir + "/f2" - val f3 = new FileOutputStream(f3Path) - f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f3.close() - assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") - - FileUtils.deleteDirectory(tmpDir) - } - - test("reading offset bytes of a file") { - val tmpDir2 = Files.createTempDir() - val f1Path = tmpDir2 + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) - f1.close() - - // Read first few bytes - assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") - - // Read some middle bytes - assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") - - // Read last few bytes - assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") - - // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") - - // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") - - // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") - - FileUtils.deleteDirectory(tmpDir2) - } } From dbe948d9a2198845feb7c8bfa738286db41344d9 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 14:15:41 -0700 Subject: [PATCH 033/221] Moved appropriate import files from UISuite to UtilsSuite --- core/src/test/scala/spark/UtilsSuite.scala | 4 ++++ core/src/test/scala/spark/ui/UISuite.scala | 5 ----- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 73050a1d3b..942702c826 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -3,6 +3,10 @@ package spark import org.scalatest.FunSuite import java.io.{ByteArrayOutputStream, ByteArrayInputStream} import scala.util.Random +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import java.io.{FileOutputStream, File} +import com.google.common.base.Charsets class UtilsSuite extends FunSuite { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index ef2e9c2ec4..cd7d105928 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -4,11 +4,6 @@ import org.scalatest.FunSuite import org.eclipse.jetty.server.Server import java.net.ServerSocket import scala.util.{Failure, Success, Try} -import spark.Utils -import com.google.common.io.Files -import org.apache.commons.io.FileUtils -import java.io.{FileOutputStream, File} -import com.google.common.base.Charsets class UISuite extends FunSuite { test("jetty port increases under contention") { From f5f3b272f874a3ac22f4e0b2caab9053657a9124 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 14:52:29 -0700 Subject: [PATCH 034/221] Fixed mixup of start/end, moved more import files --- core/src/main/scala/spark/Utils.scala | 4 ++-- core/src/test/scala/spark/UtilsSuite.scala | 11 +++++------ core/src/test/scala/spark/ui/UISuite.scala | 4 ++-- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1c5af2700a..1da9c9574e 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -625,8 +625,8 @@ private object Utils extends Logging { def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) val length = file.length() - val effectiveStart = math.min(length, start) - val effectiveEnd = math.max(0, end) + val effectiveEnd = math.min(length, end) + val effectiveStart = math.max(0, start) val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) val stream = new FileInputStream(file) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 942702c826..d83a0307fa 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -1,12 +1,11 @@ package spark -import org.scalatest.FunSuite -import java.io.{ByteArrayOutputStream, ByteArrayInputStream} -import scala.util.Random -import com.google.common.io.Files -import org.apache.commons.io.FileUtils -import java.io.{FileOutputStream, File} import com.google.common.base.Charsets +import com.google.common.io.Files +import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream, File} +import org.scalatest.FunSuite +import org.apache.commons.io.FileUtils +import scala.util.Random class UtilsSuite extends FunSuite { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index cd7d105928..fc0c160720 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -1,9 +1,9 @@ package spark.ui +import scala.util.{Failure, Success, Try} +import java.net.ServerSocket import org.scalatest.FunSuite import org.eclipse.jetty.server.Server -import java.net.ServerSocket -import scala.util.{Failure, Success, Try} class UISuite extends FunSuite { test("jetty port increases under contention") { From 24196c91f0bd560c08399eed3cb1903248ca497a Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 15:27:52 -0700 Subject: [PATCH 035/221] Changed buffer to 10,000 bytes, created scrollbar for fixed-height log --- .../main/scala/spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- .../src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index dae9995779..27549e3b4a 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.memory} {executor.state} - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index a9c6c6d519..313ecec084 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -88,9 +88,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 1f4ba5de60..490fe15fcf 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -118,9 +118,9 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option {backButton} {nextButton}

    -
    {logText}
    - {backButton} - {nextButton} +
    +
    {logText}
    +
    UIUtils.basicSparkPage(content, "Log Page for " + appId) From 74bd3fc6806a4185a95a19c107e1a4776582db44 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 15:44:28 -0700 Subject: [PATCH 036/221] Added byte range on log pages --- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 490fe15fcf..cac281eef6 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -86,7 +86,10 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option else if (offset > logLength) logLength else offset - val logText = {Utils.offsetBytes(path, fixedOffset, fixedOffset+logPageLength)} + val endOffset = math.min(fixedOffset+logPageLength, logLength) + + val range =

    Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength}

    + val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} val backButton = if (fixedOffset > 0) { @@ -101,9 +104,9 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } val nextButton = - if (fixedOffset+logPageLength < logLength) { + if (endOffset < logLength) { + format(appId, executorId, logType, endOffset, logPageLength)}> } @@ -114,6 +117,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val content = + {range}
    {backButton} {nextButton} From ebf5b8c7c5fe72fbe2d74a1bfa67cbdc9f9be0ea Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Wed, 10 Jul 2013 22:16:06 -0700 Subject: [PATCH 037/221] Updating README to reflect Scala 2.9.3 requirements --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index ba24ab43b1..1dd96a0a4a 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,7 @@ This README file only contains basic setup instructions. ## Building -Spark requires Scala 2.9.2 (Scala 2.10 is not yet supported). The project is +Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The project is built using Simple Build Tool (SBT), which is packaged with it. To build Spark and its example programs, run: From 0ecc33f0c80733ff88518157be543ec458a76bdb Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 11:25:58 -0700 Subject: [PATCH 038/221] Added byte range, page title with log name, previous/next bytes buttons, initialization to end of log, large default buffer, buggy back to master link --- .../deploy/master/ui/ApplicationPage.scala | 4 +- .../scala/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/ui/IndexPage.scala | 9 +++-- .../spark/deploy/worker/ui/WorkerWebUI.scala | 37 +++++++++++-------- 4 files changed, 30 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 27549e3b4a..b2589abb89 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.memory} {executor.state} - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 6ae1cef940..f20ea42d7f 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -77,7 +77,7 @@ private[spark] class Worker( sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() - webUi = new WorkerWebUI(self, workDir, Some(webUiPort)) + webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 313ecec084..fc1ec31b4d 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -16,17 +16,18 @@ import spark.Utils import spark.ui.UIUtils private[spark] class IndexPage(parent: WorkerWebUI) { + val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout def renderJson(request: HttpServletRequest): JValue = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") @@ -88,9 +89,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index cac281eef6..3ab0f2eded 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -9,6 +9,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} +import spark.deploy.worker.Worker import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ @@ -18,7 +19,7 @@ import spark.ui.UIUtils * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option[Int] = None) +class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) extends Logging { implicit val timeout = Timeout( Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) @@ -71,14 +72,14 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0L) val maxBytes = 1024 * 1024 - val defaultBytes = 100 * 1024 + val defaultBytes = 10000 val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val logLength = new File(path).length() + val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(logLength-10000) val logPageLength = math.min(byteLength, maxBytes) val fixedOffset = @@ -88,46 +89,52 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val endOffset = math.min(fixedOffset+logPageLength, logLength) - val range =

    Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength}

    - val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} + val linkToMaster =

    Back to Master

    + + val range = Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength} val backButton = if (fixedOffset > 0) { - + } else { - + } val nextButton = if (endOffset < logLength) { - + } else { - + } + val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} + val content = - {range} -
    - {backButton} - {nextButton} -

    + {linkToMaster} +
    +
    +
    {backButton}
    +
    {range}
    +
    {nextButton}
    +
    +
    {logText}
    - UIUtils.basicSparkPage(content, "Log Page for " + appId) + UIUtils.basicSparkPage(content, logType + " log Page for " + appId) } def stop() { From 044d4577ec564fe0f0bd9c1ad86f4749b50fc100 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 12:02:15 -0700 Subject: [PATCH 039/221] Fixed capitalization of log page --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 3ab0f2eded..923b835c3f 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -134,7 +134,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - UIUtils.basicSparkPage(content, logType + " log Page for " + appId) + UIUtils.basicSparkPage(content, logType + " log page for " + appId) } def stop() { From e3a3fcf61b117d63db33ee28928dfd77cfd935b8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 12:16:38 -0700 Subject: [PATCH 040/221] Scrollbar on log pages appear automatically --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 923b835c3f..a3cbe4f5d3 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -129,7 +129,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
    {nextButton}

    -
    +
    {logText}
    From 11872888ca44d0c08c157973a8e35d344b0119e4 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 14:56:37 -0700 Subject: [PATCH 041/221] Created getByteRange function for logs and log pages, removed lastNBytes function --- core/src/main/scala/spark/Utils.scala | 30 +++++++++--- .../spark/deploy/worker/ui/WorkerWebUI.scala | 47 +++++++++---------- 2 files changed, 45 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1da9c9574e..04041d1179 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,6 +621,29 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } + /** Determine the byte range for a log or log page. */ + def getByteRange(path: String, offset: Option[String], byteLength: Option[String]) + : (Long, Long, Long, Int) = { + val defaultBytes = 10000 + val maxBytes = 1024 * 1024 + + val file = new File(path) + val logLength = file.length() + val getOffset = offset.map(_.toLong).getOrElse(logLength-defaultBytes) + + val fixedOffset = + if (getOffset < 0) 0L + else if (getOffset > logLength) logLength + else getOffset + + val getByteLength = byteLength.map(_.toInt).getOrElse(defaultBytes) + val logPageLength = math.min(getByteLength, maxBytes) + + val endOffset = math.min(fixedOffset+logPageLength, logLength) + + (fixedOffset, endOffset, logLength, logPageLength) + } + /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) @@ -636,13 +659,6 @@ private object Utils extends Logging { Source.fromBytes(buff).mkString } - /** Return a string containing the last `n` bytes of a file. */ - def lastNBytes(path: String, n: Int): String = { - val file = new File(path) - val length = file.length() - offsetBytes(path, length-n, length) - } - /** * Clone an object using a Spark serializer. */ diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index a3cbe4f5d3..f8ac682dbf 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -57,41 +57,39 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - - val maxBytes = 1024 * 1024 // Guard against OOM - val defaultBytes = 100 * 1024 - val numBytes = Option(request.getParameter("numBytes")) - .flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) - + val offset = Option(request.getParameter("offset")) + val byteLength = Option(request.getParameter("byteLength")) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val pre = "==== Last %s bytes of %s/%s/%s ====\n".format(numBytes, appId, executorId, logType) - pre + Utils.lastNBytes(path, math.min(numBytes, maxBytes)) + + val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val fixedOffset = offsetBytes._1 + val endOffset = offsetBytes._2 + val logLength = offsetBytes._3 + + val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n" + .format(fixedOffset, endOffset, logLength, appId, executorId, logType) + pre + Utils.offsetBytes(path, fixedOffset, endOffset) } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - - val maxBytes = 1024 * 1024 - val defaultBytes = 10000 - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - + val offset = Option(request.getParameter("offset")) + val byteLength = Option(request.getParameter("byteLength")) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val logLength = new File(path).length() - val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(logLength-10000) - val logPageLength = math.min(byteLength, maxBytes) - val fixedOffset = - if (offset < 0) 0 - else if (offset > logLength) logLength - else offset + val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val fixedOffset = offsetBytes._1 + val endOffset = offsetBytes._2 + val logLength = offsetBytes._3 + val logPageLength = offsetBytes._4 - val endOffset = math.min(fixedOffset+logPageLength, logLength) + val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} val linkToMaster =

    Back to Master

    - val range = Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength} + val range = Bytes {fixedOffset.toString} - {endOffset.toString} of {logLength} val backButton = if (fixedOffset > 0) { @@ -116,8 +114,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } - val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} - val content = @@ -134,7 +130,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
    - UIUtils.basicSparkPage(content, logType + " log page for " + appId) + UIUtils.basicSparkPage(content, request.getParameter("logType") + " log page for " + + request.getParameter("appId")) } def stop() { From 15fd11d65735265f6fcc25419b43c8c104a4bd17 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:01:50 -0700 Subject: [PATCH 042/221] Removed redundant calls to request by logPage --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index f8ac682dbf..676de231f0 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -130,8 +130,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - UIUtils.basicSparkPage(content, request.getParameter("logType") + " log page for " + - request.getParameter("appId")) + UIUtils.basicSparkPage(content, logType + " log page for " + appId) } def stop() { From 5d5dbc39f603201a3596031542d52742cfb30139 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:22:45 -0700 Subject: [PATCH 043/221] getByteRange moved to WorkerWebUI, takes converted parameters, returns only start/end offset --- core/src/main/scala/spark/Utils.scala | 23 ---------- .../spark/deploy/worker/ui/WorkerWebUI.scala | 43 +++++++++++++++---- 2 files changed, 34 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 04041d1179..5a37532306 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,29 +621,6 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } - /** Determine the byte range for a log or log page. */ - def getByteRange(path: String, offset: Option[String], byteLength: Option[String]) - : (Long, Long, Long, Int) = { - val defaultBytes = 10000 - val maxBytes = 1024 * 1024 - - val file = new File(path) - val logLength = file.length() - val getOffset = offset.map(_.toLong).getOrElse(logLength-defaultBytes) - - val fixedOffset = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset - - val getByteLength = byteLength.map(_.toInt).getOrElse(defaultBytes) - val logPageLength = math.min(getByteLength, maxBytes) - - val endOffset = math.min(fixedOffset+logPageLength, logLength) - - (fixedOffset, endOffset, logLength, logPageLength) - } - /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 676de231f0..1fb59de1d8 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -57,14 +57,15 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")) - val byteLength = Option(request.getParameter("byteLength")) + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val offsetBytes = getByteRange(path, offset, byteLength) val fixedOffset = offsetBytes._1 val endOffset = offsetBytes._2 - val logLength = offsetBytes._3 + val file = new File(path) + val logLength = file.length val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n" .format(fixedOffset, endOffset, logLength, appId, executorId, logType) @@ -75,15 +76,16 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")) - val byteLength = Option(request.getParameter("byteLength")) + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val offsetBytes = getByteRange(path, offset, byteLength) val fixedOffset = offsetBytes._1 val endOffset = offsetBytes._2 - val logLength = offsetBytes._3 - val logPageLength = offsetBytes._4 + val file = new File(path) + val logLength = file.length + val logPageLength = endOffset-fixedOffset val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} @@ -133,6 +135,29 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I UIUtils.basicSparkPage(content, logType + " log page for " + appId) } + /** Determine the byte range for a log or log page. */ + def getByteRange(path: String, offset: Option[Long], byteLength: Option[Int]) + : (Long, Long) = { + val defaultBytes = 10000 + val maxBytes = 1024 * 1024 + + val file = new File(path) + val logLength = file.length() + val getOffset = offset.getOrElse(logLength-defaultBytes) + + val fixedOffset = + if (getOffset < 0) 0L + else if (getOffset > logLength) logLength + else getOffset + + val getByteLength = byteLength.getOrElse(defaultBytes) + val logPageLength = math.min(getByteLength, maxBytes) + + val endOffset = math.min(fixedOffset+logPageLength, logLength) + + (fixedOffset, endOffset) + } + def stop() { server.foreach(_.stop()) } From fdc226a14cfdb5f699627191f1682763b8571126 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:36:43 -0700 Subject: [PATCH 044/221] Clarified start and end byte variable names --- .../spark/deploy/worker/ui/WorkerWebUI.scala | 36 +++++++++---------- 1 file changed, 16 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 1fb59de1d8..2a82c35231 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -61,15 +61,13 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = getByteRange(path, offset, byteLength) - val fixedOffset = offsetBytes._1 - val endOffset = offsetBytes._2 + val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n" - .format(fixedOffset, endOffset, logLength, appId, executorId, logType) - pre + Utils.offsetBytes(path, fixedOffset, endOffset) + .format(startByte, endByte, logLength, appId, executorId, logType) + pre + Utils.offsetBytes(path, startByte, endByte) } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { @@ -80,25 +78,23 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = getByteRange(path, offset, byteLength) - val fixedOffset = offsetBytes._1 - val endOffset = offsetBytes._2 + val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - val logPageLength = endOffset-fixedOffset + val logPageLength = endByte-startByte - val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} + val logText = {Utils.offsetBytes(path, startByte, endByte)} val linkToMaster =

    Back to Master

    - val range = Bytes {fixedOffset.toString} - {endOffset.toString} of {logLength} + val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} val backButton = - if (fixedOffset > 0) { + if (startByte > 0) { - + } else { @@ -106,10 +102,10 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } val nextButton = - if (endOffset < logLength) { + if (endByte < logLength) { - + format(appId, executorId, logType, endByte, logPageLength)}> + } else { @@ -145,7 +141,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val logLength = file.length() val getOffset = offset.getOrElse(logLength-defaultBytes) - val fixedOffset = + val startByte = if (getOffset < 0) 0L else if (getOffset > logLength) logLength else getOffset @@ -153,9 +149,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val getByteLength = byteLength.getOrElse(defaultBytes) val logPageLength = math.min(getByteLength, maxBytes) - val endOffset = math.min(fixedOffset+logPageLength, logLength) + val endByte = math.min(startByte+logPageLength, logLength) - (fixedOffset, endOffset) + (startByte, endByte) } def stop() { From 9ed036ccdbb1c7aa3279998f3177c1be6b01e16b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:33:53 -0700 Subject: [PATCH 045/221] Replaced logPageLength with byteLength to prevent buffer shrink bug --- .../spark/deploy/worker/ui/WorkerWebUI.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 2a82c35231..94d919f57a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -54,11 +54,12 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def log(request: HttpServletRequest): String = { + val defaultBytes = 10000 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val (startByte, endByte) = getByteRange(path, offset, byteLength) @@ -71,17 +72,18 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { + val defaultBytes = 10000 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - val logPageLength = endByte-startByte + //val logPageLength = endByte-startByte val logText = {Utils.offsetBytes(path, startByte, endByte)} @@ -92,9 +94,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val backButton = if (startByte > 0) { - + .format(appId, executorId, logType, math.max(startByte-byteLength, 0), + byteLength)}> + } else { @@ -104,8 +106,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val nextButton = if (endByte < logLength) { - + format(appId, executorId, logType, endByte, byteLength)}> + } else { @@ -132,7 +134,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } /** Determine the byte range for a log or log page. */ - def getByteRange(path: String, offset: Option[Long], byteLength: Option[Int]) + def getByteRange(path: String, offset: Option[Long], byteLength: Int) : (Long, Long) = { val defaultBytes = 10000 val maxBytes = 1024 * 1024 @@ -146,8 +148,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I else if (getOffset > logLength) logLength else getOffset - val getByteLength = byteLength.getOrElse(defaultBytes) - val logPageLength = math.min(getByteLength, maxBytes) + val logPageLength = math.min(byteLength, maxBytes) val endByte = math.min(startByte+logPageLength, logLength) From ece2388585a468dd57c973f254ba20c24299a464 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:35:56 -0700 Subject: [PATCH 046/221] Removed logPageLength from logPage --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 94d919f57a..1ab65d41e8 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -83,7 +83,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - //val logPageLength = endByte-startByte val logText = {Utils.offsetBytes(path, startByte, endByte)} From a32784109d11e061cdca2eced07ac01be2061056 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:57:55 -0700 Subject: [PATCH 047/221] Fixed links for "Back to Master" --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index dbdc8e1057..4dd6c448a9 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -71,7 +71,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } else { addWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) context.watch(sender) // This doesn't work with remote actors but helps for testing - sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort) + sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort.get) schedule() } } From 6d054487bf4c9e9aa9033187f73e4e9d8366bc37 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 17:12:17 -0700 Subject: [PATCH 048/221] Replace default buffer value to 100 GB, changed buttons to use String notation, removed default buffer parameter in UI URLs --- .../spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- .../scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 12 +++++++----- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index b2589abb89..8553377d8f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.memory} {executor.state} - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index fc1ec31b4d..c515f2e238 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -89,9 +89,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 1ab65d41e8..4e7f86d77a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -54,7 +54,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def log(request: HttpServletRequest): String = { - val defaultBytes = 10000 + val defaultBytes = 100 * 1024 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") @@ -72,7 +72,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { - val defaultBytes = 10000 + val defaultBytes = 100 * 1024 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") @@ -95,7 +95,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - + } else { @@ -106,7 +107,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I if (endByte < logLength) { - + } else { @@ -135,7 +137,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I /** Determine the byte range for a log or log page. */ def getByteRange(path: String, offset: Option[Long], byteLength: Int) : (Long, Long) = { - val defaultBytes = 10000 + val defaultBytes = 100 * 1024 val maxBytes = 1024 * 1024 val file = new File(path) From 5c67ca027834582bcd6cdd55baee5fd74e743a71 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 17:31:59 -0700 Subject: [PATCH 049/221] Remove "Bytes" in lieu of String notation --- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 4e7f86d77a..ccd55c1ce4 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -95,24 +95,22 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - + } else { - + } val nextButton = if (endByte < logLength) { - + } else { - + } val content = From a2c915fba84a1deadac199994322e0f15d5750b4 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 11 Jul 2013 18:55:00 -0700 Subject: [PATCH 050/221] giving order to top and making tests more clear --- core/src/main/scala/spark/RDD.scala | 6 ++---- core/src/test/scala/spark/RDDSuite.scala | 8 ++++---- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index af52040fa6..4c1591ed5a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -778,7 +778,7 @@ abstract class RDD[T: ClassManifest]( }.reduce { (queue1, queue2) => queue1 ++= queue2 queue1 - }.toArray + }.toArray.sorted(ord.reverse) } /** @@ -789,9 +789,7 @@ abstract class RDD[T: ClassManifest]( * @param ord the implicit ordering for T * @return an array of top elements */ - def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { - top(num)(ord.reverse).sorted(ord) - } + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse) /** * Save this RDD as a text file, using string representations of elements. diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index fe17d1d5e7..aa3ee5f5ee 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -240,7 +240,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) val topK = ints.top(5) assert(topK.size === 5) - assert(topK.sorted === nums.sorted.takeRight(5)) + assert(topK === nums.reverse.take(5)) } test("top with custom ordering") { @@ -255,9 +255,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("takeOrdered with predefined ordering") { val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) val rdd = sc.makeRDD(nums, 2) - val sortedTopK = rdd.takeOrdered(5) - assert(sortedTopK.size === 5) - assert(sortedTopK === Array(1, 2, 3, 4, 5)) + val sortedLowerK = rdd.takeOrdered(5) + assert(sortedLowerK.size === 5) + assert(sortedLowerK === Array(1, 2, 3, 4, 5)) } test("takeOrdered with custom ordering") { From a1662326e9b1486361eba2f2caf903875fbba597 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 12 Jul 2013 08:38:19 -0700 Subject: [PATCH 051/221] comment adjustment to takeOrdered --- core/src/main/scala/spark/RDD.scala | 2 +- core/src/main/scala/spark/api/java/JavaRDDLike.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 4c1591ed5a..8aa77266bc 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -782,7 +782,7 @@ abstract class RDD[T: ClassManifest]( } /** - * Returns the top K elements from this RDD as defined by + * Returns the first K elements from this RDD as defined by * the specified implicit Ordering[T] and maintains the * ordering. * @param num the number of top elements to return diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 94b95af714..27f40ecdfd 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -386,7 +386,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the top K elements from this RDD as defined by + * Returns the first K elements from this RDD as defined by * the specified Comparator[T] and maintains the order. * @param num the number of top elements to return * @param comp the comparator that defines the order @@ -400,7 +400,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the top K elements from this RDD using the + * Returns the first K elements from this RDD using the * natural ordering for T while maintain the order. * @param num the number of top elements to return * @return an array of top elements From cd7259b4b8d8abbff6db963fd8f84d4bd0b3737b Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Fri, 12 Jul 2013 11:51:14 -0700 Subject: [PATCH 052/221] Fixes typos in Spark Streaming Programming Guide These typos were reported on the spark-users mailing list, see: https://groups.google.com/d/msg/spark-users/SyLGgJlKCrI/LpeBypOkSMUJ --- docs/streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index f5788dc467..8cd1b0cd66 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -7,7 +7,7 @@ title: Spark Streaming Programming Guide {:toc} # Overview -A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transformong existing DStreams using parallel operators like `map`, `reduce`, and `window`. The basic processing model is as follows: +A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transforming existing DStreams using parallel operators like `map`, `reduce`, and `window`. The basic processing model is as follows: (i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides it into batches. Each batch of data is treated as an RDD, that is, an immutable parallel collection of data. These input RDDs are saved in memory and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively called an InputDStream. (ii) Data received by InputDStreams are processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures. @@ -20,7 +20,7 @@ The first thing a Spark Streaming program must do is create a `StreamingContext` new StreamingContext(master, appName, batchDuration, [sparkHome], [jars]) {% endhighlight %} -The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). +The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such that the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). This constructor creates a SparkContext for your job as well, which can be accessed with `streamingContext.sparkContext`. From 531a7e55745ad76d17d41f80dbdea0072acce079 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 10 Jul 2013 17:59:43 -0700 Subject: [PATCH 053/221] Pass executor env vars (e.g. SPARK_CLASSPATH) to compute-classpath. --- core/src/main/scala/spark/Utils.scala | 12 +++++++++--- .../scala/spark/deploy/worker/ExecutorRunner.scala | 4 +++- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 64547bbdcd..d2bf151cbf 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -6,6 +6,7 @@ import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} import java.util.regex.Pattern +import scala.collection.Map import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConversions._ import scala.io.Source @@ -545,10 +546,15 @@ private object Utils extends Logging { /** * Execute a command and get its output, throwing an exception if it yields a code other than 0. */ - def executeAndGetOutput(command: Seq[String], workingDir: File = new File(".")): String = { - val process = new ProcessBuilder(command: _*) + def executeAndGetOutput(command: Seq[String], workingDir: File = new File("."), + extraEnvironment: Map[String, String] = Map.empty): String = { + val builder = new ProcessBuilder(command: _*) .directory(workingDir) - .start() + val environment = builder.environment() + for ((key, value) <- extraEnvironment) { + environment.put(key, value) + } + val process = builder.start() new Thread("read stderr for " + command(0)) { override def run() { for (line <- Source.fromInputStream(process.getErrorStream).getLines) { diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index d7f58b2cb1..5d3d54c65e 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -98,7 +98,9 @@ private[spark] class ExecutorRunner( // 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)) + val classPath = Utils.executeAndGetOutput( + Seq(sparkHome + "/bin/compute-classpath" + ext), + extraEnvironment=appDesc.command.environment) Seq("-cp", classPath) ++ libraryOpts ++ userOpts ++ memoryOpts } From 73984b96a8450674b472676eaa855cc2df68a754 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 12 Jul 2013 14:26:56 -0700 Subject: [PATCH 054/221] Removed unit test of nonexistent function Utils.lastNBytes --- core/src/test/scala/spark/UtilsSuite.scala | 27 ---------------------- 1 file changed, 27 deletions(-) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index d83a0307fa..1e1260f606 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -91,33 +91,6 @@ class UtilsSuite extends FunSuite { assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") } - test("reading last n bytes of a file") { - val tmpDir = Files.createTempDir() - - // File smaller than limit - val f1Path = tmpDir + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) - f1.close() - assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") - - // File larger than limit - val f2Path = tmpDir + "/f2" - val f2 = new FileOutputStream(f2Path) - f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f2.close() - assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") - - // Request limit too - val f3Path = tmpDir + "/f2" - val f3 = new FileOutputStream(f3Path) - f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f3.close() - assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") - - FileUtils.deleteDirectory(tmpDir) - } - test("reading offset bytes of a file") { val tmpDir2 = Files.createTempDir() val f1Path = tmpDir2 + "/f1" From 6855338e1400638188358a7d7926eb86f668c160 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Jul 2013 19:24:16 -0700 Subject: [PATCH 055/221] Show block locations in Web UI. This fixes SPARK-769. Support is added for enumerating the locations of blocks in the UI. There is also some minor cleanup in StorageUtils. --- .../main/scala/spark/storage/StorageUtils.scala | 11 +++++++++-- .../main/scala/spark/ui/storage/RDDPage.scala | 17 +++++++++++++---- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 950c0cdf35..3e7fa287e5 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -39,12 +39,19 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, private[spark] object StorageUtils { - /* Given the current storage status of the BlockManager, returns information for each RDD */ - def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus], + /* Returns RDD-level information, compiled from a list of StorageStatus objects */ + def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus], sc: SparkContext) : Array[RDDInfo] = { rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) } + /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ + def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { + val blockLocationPairs = storageStatusList + .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) + blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap + } + /* Given a list of BlockStatus objets, returns information for each RDD */ def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], sc: SparkContext) : Array[RDDInfo] = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 0cb1e47ea5..428db6fa95 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -26,8 +26,14 @@ private[spark] class RDDPage(parent: BlockManagerUI) { val workers = filteredStorageStatusList.map((prefix, _)) val workerTable = listingTable(workerHeaders, workerRow, workers) - val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk") - val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk", + "Locations") + + val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) + val blocks = blockStatuses.map { + case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN"))) + } val blockTable = listingTable(blockHeaders, blockRow, blocks) val content = @@ -74,8 +80,8 @@ private[spark] class RDDPage(parent: BlockManagerUI) { headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name, Jobs) } - def blockRow(blk: (String, BlockStatus)): Seq[Node] = { - val (id, block) = blk + def blockRow(row: (String, BlockStatus, Seq[String])): Seq[Node] = { + val (id, block, locations) = row {id} @@ -87,6 +93,9 @@ private[spark] class RDDPage(parent: BlockManagerUI) { {Utils.memoryBytesToString(block.diskSize)} + + {locations.map(l => {l}
    )} + } From 08150f19abcf08f2a18305080b08736fb8a33a12 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Jul 2013 19:32:35 -0700 Subject: [PATCH 056/221] Minor style fix --- core/src/main/scala/spark/storage/StorageUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 3e7fa287e5..aca16f533a 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -47,8 +47,8 @@ object StorageUtils { /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { - val blockLocationPairs = storageStatusList - .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) + val blockLocationPairs = storageStatusList + .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap } From c4d5b01e44c5f289d67670185a14ff4ccd9537db Mon Sep 17 00:00:00 2001 From: seanm Date: Sat, 13 Jul 2013 14:56:23 -0600 Subject: [PATCH 057/221] changing com.google.code.findbugs maven coordinates --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 641d379749..0a4c176a29 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -144,7 +144,7 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", - "com.google.code.findbugs" % "jsr305" % "1.3.+", + "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.16", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, From 1889f9f450d3e28864a5ac7b5f9d3831cf7b17bf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 13 Jul 2013 14:43:21 -0700 Subject: [PATCH 058/221] Increase PermGen size --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 850c58e1e9..523fbb346b 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -5,4 +5,4 @@ if [ "$MESOS_HOME" != "" ]; then fi export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) export SPARK_TESTING=1 # To put test classes on classpath -java -Xmx1200m -XX:MaxPermSize=250m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" From ddb97f0fdf16442afaa9cab656376267e4044510 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 13 Jul 2013 16:05:21 -0700 Subject: [PATCH 059/221] Add `Environment` tab to SparkUI. This adds a tab which displays system property and classpath information. This can be useful in debugging various types of issues such as: 1. Extra/incorrect Hadoop jars being included in the classpath 2. Spark launching with a different JRE version than intended 3. Spark system properties not being set to intended values 4. User added jars that conflict with Spark jars --- core/src/main/scala/spark/ui/Page.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 7 +-- core/src/main/scala/spark/ui/UIUtils.scala | 5 ++ .../scala/spark/ui/env/EnvironmentUI.scala | 58 +++++++++++++++++++ 4 files changed, 67 insertions(+), 5 deletions(-) create mode 100644 core/src/main/scala/spark/ui/env/EnvironmentUI.scala diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala index c853b44b76..ed8f91842c 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/spark/ui/Page.scala @@ -1,3 +1,3 @@ package spark.ui -private[spark] object Page extends Enumeration { val Storage, Jobs = Value } \ No newline at end of file +private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index b3bdc2c490..874e5ba8ec 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -5,15 +5,13 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, SparkContext, Utils} +import spark.ui.env.EnvironmentUI import spark.ui.storage.BlockManagerUI import spark.ui.jobs.JobProgressUI -import spark.ui.UIUtils._ import spark.ui.JettyUtils._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { - // TODO(pwendell): It would be nice to add a view that prints out environment information - val host = Utils.localHostName() val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None @@ -25,7 +23,8 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) - val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers + val env = new EnvironmentUI(sc) + val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ handlers /** Bind the HTTP server which backs this web interface */ def bind() { diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index 7b79290d1b..36d9c47245 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -19,6 +19,10 @@ private[spark] object UIUtils { case Jobs =>
  • Jobs
  • case _ =>
  • Jobs
  • } + val environment = page match { + case Environment =>
  • Environment
  • + case _ =>
  • Environment
  • + } @@ -44,6 +48,7 @@ private[spark] object UIUtils {
    • Application: {sc.appName}
    • diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala new file mode 100644 index 0000000000..8a3b330405 --- /dev/null +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -0,0 +1,58 @@ +package spark.ui.env + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.collection.JavaConversions._ + +import spark.ui.JettyUtils._ +import spark.ui.UIUtils.headerSparkPage +import spark.ui.Page.Environment +import spark.SparkContext +import spark.ui.UIUtils + +import scala.xml.Node + +private[spark] class EnvironmentUI(sc: SparkContext) { + + def getHandlers = Seq[(String, Handler)]( + ("/environment", (request: HttpServletRequest) => envDetails(request)) + ) + + def envDetails(request: HttpServletRequest): Seq[Node] = { + val properties = System.getProperties.iterator.toSeq + + val classPathProperty = properties + .filter{case (k, v) => k.contains("java.class.path")} + .headOption + .getOrElse("", "") + val sparkProperties = properties.filter(_._1.contains("spark")) + val otherProperties = properties.diff(sparkProperties :+ classPathProperty) + + val propertyHeaders = Seq("Name", "Value") + def propertyRow(kv: (String, String)) = {kv._1}{kv._2} + val propertyTable = UIUtils.listingTable( + propertyHeaders, propertyRow, sparkProperties ++ otherProperties) + + val classPathEntries = classPathProperty._2 + .split(System.getProperty("path.separator", ":")) + .filterNot(e => e.isEmpty) + .map(e => (e, "System Classpath")) + val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} + val addedFiles = sc.addedFiles.iterator.toSeq.map{case (path, time) => (path, "Added By User")} + val classPath = addedJars ++ addedFiles ++ classPathEntries + + val classPathHeaders = Seq("Resource", "Source") + def classPathRow(data: (String, String)) = {data._1}{data._2} + val classPathTable = UIUtils.listingTable(classPathHeaders, classPathRow, classPath) + + val content = + +

      System Properties

      {propertyTable} +

      Classpath Entries

      {classPathTable} +
      + + headerSparkPage(content, sc, "Environment", Environment) + } +} From e271fde10b342216e33f4f45af73c5d103215cf2 Mon Sep 17 00:00:00 2001 From: root Date: Sun, 14 Jul 2013 06:24:29 +0000 Subject: [PATCH 060/221] Fixed a delay scheduling bug in the YARN branch, found by Patrick --- .../scheduler/cluster/ClusterTaskSetManager.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index fe6420a522..327d6797ae 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -454,10 +454,10 @@ private[spark] class ClusterTaskSetManager( val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val taskLocality = - if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else - if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else - if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else - TaskLocality.ANY + if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL + else if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL + else if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL + else TaskLocality.ANY val prefStr = taskLocality.toString logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( taskSet.id, index, taskId, execId, hostPort, prefStr)) @@ -467,7 +467,7 @@ private[spark] class ClusterTaskSetManager( val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) - if (TaskLocality.NODE_LOCAL == taskLocality) { + if (taskLocality == TaskLocality.PROCESS_LOCAL || taskLocality == TaskLocality.NODE_LOCAL) { lastPreferredLaunchTime = time } // Serialize and return the task From a44a7b123862202cf97f2de7a96aeaf29a93002a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:23:09 +0000 Subject: [PATCH 061/221] Determine Spark core classes better in getCallSite --- core/src/main/scala/spark/Utils.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c6a3f97872..a36186bf8a 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -579,8 +579,15 @@ private object Utils extends Logging { output.toString } + /** + * A regular expression to match classes of the "core" Spark API that we want to skip when + * finding the call site of a method. + */ + private val SPARK_CLASS_REGEX = """^spark(\.api\.java)?(\.rdd)?\.[A-Z]""".r + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, val firstUserLine: Int, val firstUserClass: String) + /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. @@ -602,7 +609,7 @@ private object Utils extends Logging { for (el <- trace) { if (!finished) { - if (el.getClassName.startsWith("spark.") && !el.getClassName.startsWith("spark.examples.")) { + if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName) != None) { lastSparkMethod = if (el.getMethodName == "") { // Spark method is a constructor; get its class name el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) From b91a218cea5a7ab4037675667922fc06bfec6fbf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:31:33 +0000 Subject: [PATCH 062/221] Cosmetic fixes to web UI --- .../src/main/scala/spark/deploy/master/ui/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index c6de2bafa3..7545ecf868 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -38,8 +38,8 @@ private[spark] class IndexPage(parent: MasterWebUI) {
        -
      • URL:{state.uri}
      • -
      • Workers:{state.workers.size}
      • +
      • URL: {state.uri}
      • +
      • Workers: {state.workers.size}
      • Cores: {state.workers.map(_.cores).sum} Total, {state.workers.map(_.coresUsed).sum} Used
      • Memory: diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 1e675ab2cb..7907ab3bc7 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -29,10 +29,10 @@ private[spark] class IndexPage(parent: JobProgressUI) { Stage Id Origin Submitted - Duration - Tasks: Complete/Total - Shuffle Activity - Stored RDD + Duration + Tasks: Complete/Total + Shuffle Activity + Stored RDD {rows.map(r => makeRow(r))} From c5c38d1987137a1dc5eb66dd1065735a542ef9b5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:59:50 +0000 Subject: [PATCH 063/221] Some optimizations to loading phase of ALS --- .../spark/mllib/recommendation/ALS.scala | 45 +++++++++++++------ 1 file changed, 32 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 6c9fb2359c..dec3701ec0 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -2,6 +2,7 @@ package spark.mllib.recommendation import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random +import scala.util.Sorting import spark.{HashPartitioner, Partitioner, SparkContext, RDD} import spark.storage.StorageLevel @@ -33,6 +34,12 @@ private[recommendation] case class InLinkBlock( elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[Double])]]) +/** + * A more compact class to represent a rating than Tuple3[Int, Int, Double]. + */ +private[recommendation] case class Rating(user: Int, product: Int, rating: Double) + + /** * Alternating Least Squares matrix factorization. * @@ -126,13 +133,13 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l * 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[(Int, Int, Double)]): OutLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted + private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating]): 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 ((u, p, r) <- ratings) { - shouldSend(userIdToPos(u))(p % numBlocks) = true + for (r <- ratings) { + shouldSend(userIdToPos(r.user))(r.product % numBlocks) = true } OutLinkBlock(userIds, shouldSend) } @@ -141,18 +148,28 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l * 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[(Int, Int, Double)]): InLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted + private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating]): 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 + } val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) for (productBlock <- 0 until numBlocks) { - val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) - val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) - .toArray - .sortBy(_._1) - .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} - ratingsForBlock(productBlock) = ratingsByProduct + // Create an array of (product, Seq(Rating)) ratings + val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray + // Sort them by user ID + val ordering = new Ordering[(Int, ArrayBuffer[Rating])] { + def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = a._1 - b._1 + } + Sorting.quickSort(groupedRatings)(ordering) + // Translate the user IDs to indices based on userIdToPos + ratingsForBlock(productBlock) = groupedRatings.map { case (p, rs) => + (rs.view.map(r => userIdToPos(r.user)).toArray, rs.view.map(_.rating).toArray) + } } InLinkBlock(userIds, ratingsForBlock) } @@ -167,7 +184,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l { val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { - val ratings = elements.map(_._2).toArray + val ratings = elements.map{case (k, t) => Rating(t._1, t._2, t._3)}.toArray val inLinkBlock = makeInLinkBlock(numBlocks, ratings) val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) @@ -373,6 +390,8 @@ object ALS { } val (master, ratingsFile, rank, iters, outputDir) = (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) + System.setProperty("spark.serializer", "spark.KryoSerializer") + System.setProperty("spark.locality.wait", "10000") val sc = new SparkContext(master, "ALS") val ratings = sc.textFile(ratingsFile).map { line => val fields = line.split(',') From 931e4c96ef56302cc384086d1faa0f4ca3258e8b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 08:03:13 +0000 Subject: [PATCH 064/221] Fix a comment --- mllib/src/main/scala/spark/mllib/recommendation/ALS.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index dec3701ec0..21eb21276e 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -161,7 +161,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l for (productBlock <- 0 until numBlocks) { // Create an array of (product, Seq(Rating)) ratings val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray - // Sort them by user ID + // Sort them by product ID val ordering = new Ordering[(Int, ArrayBuffer[Rating])] { def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = a._1 - b._1 } From 00556a94c98577d1536d0c2b4606a9933069a584 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Sun, 14 Jul 2013 17:04:53 +0800 Subject: [PATCH 065/221] add spaces before curly braces and after for if conditions --- .../spark/deploy/yarn/ApplicationMaster.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 68bb36d316..6a0617cc06 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -130,7 +130,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val t = new Thread { override def run() { var successed = false - try{ + try { // Copy var mainArgs: Array[String] = new Array[String](args.userArgs.size()) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) @@ -140,9 +140,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // It need shutdown hook to set SUCCEEDED successed = true } finally { - if(successed){ + if (successed) { ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - }else{ + } else { ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) } } @@ -190,7 +190,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("All workers have launched.") // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout - if (userThread.isAlive){ + if (userThread.isAlive) { // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) @@ -208,7 +208,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val t = new Thread { override def run() { - while (userThread.isAlive){ + while (userThread.isAlive) { val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning if (missingWorkerCount > 0) { logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") @@ -250,7 +250,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e def finishApplicationMaster(status: FinalApplicationStatus) { synchronized { - if(isFinished){ + if (isFinished) { return } isFinished = true @@ -276,7 +276,7 @@ object ApplicationMaster { private val ALLOCATOR_LOOP_WAIT_COUNT = 30 def incrementAllocatorLoop(by: Int) { val count = yarnAllocatorLoop.getAndAdd(by) - if (count >= ALLOCATOR_LOOP_WAIT_COUNT){ + if (count >= ALLOCATOR_LOOP_WAIT_COUNT) { yarnAllocatorLoop.synchronized { // to wake threads off wait ... yarnAllocatorLoop.notifyAll() @@ -320,7 +320,7 @@ object ApplicationMaster { // Wait for initialization to complete and atleast 'some' nodes can get allocated yarnAllocatorLoop.synchronized { - while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT){ + while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) { yarnAllocatorLoop.wait(1000L) } } From 4883586838b960c405af208262aed3ec4b078613 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 14 Jul 2013 10:37:26 -0700 Subject: [PATCH 066/221] Responding to Matei's review --- .../scala/spark/ui/env/EnvironmentUI.scala | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index 8a3b330405..6b8b9f05bb 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -5,6 +5,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler import scala.collection.JavaConversions._ +import scala.util.Properties import spark.ui.JettyUtils._ import spark.ui.UIUtils.headerSparkPage @@ -21,19 +22,27 @@ private[spark] class EnvironmentUI(sc: SparkContext) { ) def envDetails(request: HttpServletRequest): Seq[Node] = { - val properties = System.getProperties.iterator.toSeq + val jvmInformation = Seq( + ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), + ("Java Home", Properties.javaHome), + ("Scala Version", Properties.versionString), + ("Scala Home", Properties.scalaHome) + ) + def jvmRow(kv: (String, String)) = {kv._1}{kv._2} + def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation) + val properties = System.getProperties.iterator.toSeq val classPathProperty = properties .filter{case (k, v) => k.contains("java.class.path")} .headOption .getOrElse("", "") - val sparkProperties = properties.filter(_._1.contains("spark")) + val sparkProperties = properties.filter(_._1.startsWith("spark")) val otherProperties = properties.diff(sparkProperties :+ classPathProperty) val propertyHeaders = Seq("Name", "Value") def propertyRow(kv: (String, String)) = {kv._1}{kv._2} - val propertyTable = UIUtils.listingTable( - propertyHeaders, propertyRow, sparkProperties ++ otherProperties) + val sparkPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties) + val otherPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties) val classPathEntries = classPathProperty._2 .split(System.getProperty("path.separator", ":")) @@ -49,7 +58,9 @@ private[spark] class EnvironmentUI(sc: SparkContext) { val content = -

        System Properties

        {propertyTable} +

        Runtime Information

        {jvmTable} +

        Spark Properties

        {sparkPropertyTable} +

        System Properties

        {otherPropertyTable}

        Classpath Entries

        {classPathTable}
        From ed7fd501cf7ece730cbdee6c152b917cf6bfb16a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 00:30:10 +0000 Subject: [PATCH 067/221] Make number of blocks in ALS configurable and lower the default --- .../src/main/scala/spark/mllib/recommendation/ALS.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 21eb21276e..2abaf2f2dd 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -91,7 +91,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l */ def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { val numBlocks = if (this.numBlocks == -1) { - math.max(ratings.context.defaultParallelism, ratings.partitions.size) + math.max(ratings.context.defaultParallelism, ratings.partitions.size / 2) } else { this.numBlocks } @@ -384,12 +384,13 @@ object ALS { } def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: ALS ") + if (args.length != 5 && args.length != 6) { + 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 blocks = if (args.length == 6) args(5).toInt else -1 System.setProperty("spark.serializer", "spark.KryoSerializer") System.setProperty("spark.locality.wait", "10000") val sc = new SparkContext(master, "ALS") @@ -397,7 +398,7 @@ object ALS { val fields = line.split(',') (fields(0).toInt, fields(1).toInt, fields(2).toDouble) } - val model = ALS.train(ratings, rank, iters) + val model = ALS.train(ratings, rank, iters, 0.01, blocks) model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } .saveAsTextFile(outputDir + "/userFeatures") model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } From d47c16f78d5cb935bd4022c9bed8376691371682 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 01:55:54 +0000 Subject: [PATCH 068/221] Add an option to disable reference tracking in Kryo --- core/src/main/scala/spark/KryoSerializer.scala | 4 ++++ docs/configuration.md | 12 +++++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index d723ab7b1e..c7dbcc6fbc 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -210,6 +210,10 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) } + + // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops + kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) + kryo } diff --git a/docs/configuration.md b/docs/configuration.md index 5a80510959..5c06897cae 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -197,9 +197,19 @@ Apart from these, the following properties are also available, and may be useful (e.g. map functions) reference large objects in the driver program. + + spark.kryo.referenceTracking + true + + Whether to track references to the same object when serializing data with Kryo, which is + necessary if your object graphs have loops and useful for efficiency if they contain multiple + copies of the same object. Can be disabled to improve performance if you know this is not the + case. + + spark.kryoserializer.buffer.mb - 32 + 2 Maximum object size to allow within Kryo (the library needs to create a buffer at least as large as the largest single object you'll serialize). Increase this if you get a "buffer limit From 4698a0d6886905ef21cbd52e108d0dcab3df12df Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 02:54:11 +0000 Subject: [PATCH 069/221] Shuffle ratings in a more efficient way at start of ALS --- .../scala/spark/mllib/recommendation/ALS.scala | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 2abaf2f2dd..4c18cbdc6b 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -6,8 +6,10 @@ import scala.util.Sorting import spark.{HashPartitioner, Partitioner, SparkContext, RDD} import spark.storage.StorageLevel +import spark.KryoRegistrator import spark.SparkContext._ +import com.esotericsoftware.kryo.Kryo import org.jblas.{DoubleMatrix, SimpleBlas, Solve} @@ -98,8 +100,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val partitioner = new HashPartitioner(numBlocks) - val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } - val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } + val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, Rating(u, p, r)) } + val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, Rating(p, u, r)) } val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) @@ -179,12 +181,12 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l * 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, (Int, Int, Double))]) + private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)]) : (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) = { val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { - val ratings = elements.map{case (k, t) => Rating(t._1, t._2, t._3)}.toArray + val ratings = elements.map{_._2}.toArray val inLinkBlock = makeInLinkBlock(numBlocks, ratings) val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) @@ -383,6 +385,12 @@ object ALS { train(ratings, rank, iterations, 0.01, -1) } + 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 != 6) { println("Usage: ALS []") @@ -392,6 +400,8 @@ object ALS { (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) val blocks = if (args.length == 6) args(5).toInt else -1 System.setProperty("spark.serializer", "spark.KryoSerializer") + System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) + System.setProperty("spark.kryo.referenceTracking", "false") System.setProperty("spark.locality.wait", "10000") val sc = new SparkContext(master, "ALS") val ratings = sc.textFile(ratingsFile).map { line => From b2aaa1199e7ecd8e1b2a9ddd8356b6393edafe6b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 11:44:42 -0700 Subject: [PATCH 070/221] Adds app name in HTML page titles on job web UI: fixes SPARK-806 --- core/src/main/scala/spark/ui/UIUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index 36d9c47245..fa46e2487d 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -31,7 +31,7 @@ private[spark] object UIUtils { - {title} + {sc.appName} - {title} From 0d78b6d9cd11fc12c546f25fa857ba8b285c062d Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 13:42:12 -0700 Subject: [PATCH 071/221] Links to job UI from standalone deploy cluster web UI: fixes SPARK-802 --- core/src/main/scala/spark/deploy/master/Master.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 4dd6c448a9..87a7791fbd 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -80,6 +80,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logInfo("Registering app " + description.name) val app = addApplication(description, sender) logInfo("Registered app " + description.name + " with ID " + app.id) + logInfo("Started App web UI at " + description.appUiUrl) waitingApps += app context.watch(sender) // This doesn't work with remote actors but helps for testing sender ! RegisteredApplication(app.id) From 39557112501901da7f9b4be6159a5a0be5511b42 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 15:47:21 -0700 Subject: [PATCH 072/221] Added field to master UI with link to job UI --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 7545ecf868..434f600e8e 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -26,8 +26,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User", - "State", "Duration") + val appHeaders = Seq("ID", "Job UI", "Description", "Cores", "Memory per Node", "Submit Time", + "User", "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) val completedApps = state.completedApps.sortBy(_.endTime).reverse @@ -103,6 +103,9 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.id} + + {app.appUiUrl} + {app.desc.name} {app.coresGranted} From fbf5aa761e40649d0488a8673d488882c8bdae48 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 15:50:03 -0700 Subject: [PATCH 073/221] Removed log message, added field in master UI to link to log UI --- core/src/main/scala/spark/deploy/master/Master.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 87a7791fbd..4dd6c448a9 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -80,7 +80,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logInfo("Registering app " + description.name) val app = addApplication(description, sender) logInfo("Registered app " + description.name + " with ID " + app.id) - logInfo("Started App web UI at " + description.appUiUrl) waitingApps += app context.watch(sender) // This doesn't work with remote actors but helps for testing sender ! RegisteredApplication(app.id) From 6dc7c9bfb17cefdf162e86c8b52a4ffc8b59efaf Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 16:33:50 -0700 Subject: [PATCH 074/221] Removed job UI column, linked description to job UI --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 434f600e8e..5e3c5e064f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -26,8 +26,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("ID", "Job UI", "Description", "Cores", "Memory per Node", "Submit Time", - "User", "State", "Duration") + val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User", + "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) val completedApps = state.completedApps.sortBy(_.endTime).reverse @@ -104,9 +104,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.id} - {app.appUiUrl} + {app.desc.name} - {app.desc.name} {app.coresGranted} From a96b4ef761aa80310b194176f41a088c5bf6274a Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Jul 2013 19:13:17 -0600 Subject: [PATCH 075/221] dding tgz option to make-distribution.sh --- make-distribution.sh | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index feb13d52f9..ef3d2529d0 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Script to create a binary distribution for easy deploys of Spark. # The distribution directory defaults to dist/ but can be overridden below. @@ -6,6 +6,10 @@ # so it is completely self contained. # It does not contain source or *.class files. # +# Arguments +# (none): Creates dist/ directory +# tgz: Additionally creates spark-$VERSION-bin.tar.gz +# # Recommended deploy/testing procedure (standalone mode): # 1) Rsync / deploy the dist/ dir to one host # 2) cd to deploy dir; ./bin/start-master.sh @@ -19,8 +23,14 @@ DISTDIR="$FWDIR/dist" # Get version from SBT export TERM=dumb # Prevents color codes in SBT output -VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2) -echo "Making distribution for Spark $VERSION in $DISTDIR..." +VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') + +if [ "$1" == "tgz" ]; then + echo "Making spark-$VERSION-bin.tar.gz" +else + echo "Making distribution for Spark $VERSION in $DISTDIR..." +fi + # Build fat JAR $FWDIR/sbt/sbt "repl/assembly" @@ -37,3 +47,11 @@ cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/conf" "$DISTDIR" cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" + + +if [ "$1" == "tgz" ]; then + TARDIR="$FWDIR/spark-$VERSION" + cp -r $DISTDIR $TARDIR + tar -zcf spark-$VERSION-bin.tar.gz -C $FWDIR spark-$VERSION + rm -rf $TARDIR +fi From 90b0142985ca2c6c76ded1a5c073774308c1a727 Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Jul 2013 19:13:39 -0600 Subject: [PATCH 076/221] adding files generated from make-distribution.sh to .gitignore --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index ae39c52b11..00fbff6a2c 100644 --- a/.gitignore +++ b/.gitignore @@ -38,3 +38,5 @@ dependency-reduced-pom.xml .ensime_lucene checkpoint derby.log +dist/ +spark-*-bin.tar.gz From 69316603d6bf11ecf1ea3dab63df178bad835e2d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 15 Jul 2013 22:50:11 -0700 Subject: [PATCH 077/221] Throw a more meaningful message when runJob is called to launch tasks on non-existent partitions. --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 9 +++++++++ core/src/test/scala/spark/RDDSuite.scala | 6 ++++++ 2 files changed, 15 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 3d3b9ea011..8173ef709d 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -251,6 +251,15 @@ class DAGScheduler( if (partitions.size == 0) { return } + + // Check to make sure we are not launching a task on a partition that does not exist. + val maxPartitions = finalRdd.partitions.length + partitions.find(p => p >= maxPartitions).foreach { p => + throw new IllegalArgumentException( + "Attempting to access a non-existent partition: " + p + ". " + + "Total number of partitions: " + maxPartitions) + } + val (toSubmit, waiter) = prepareJob( finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index aa3ee5f5ee..7f7d4c8211 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -302,4 +302,10 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") } } + + test("runJob on an invalid partition") { + intercept[IllegalArgumentException] { + sc.runJob(sc.parallelize(1 to 10, 2), {iter: Iterator[Int] => iter.size}, Seq(0, 1, 2), false) + } + } } From 2748e73eb9bb3f08b116c58ea404a6e76f9cbff9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 16:08:46 +0530 Subject: [PATCH 078/221] Dependency upgrade Akka 2.0.3 -> 2.0.5 --- project/SparkBuild.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 178a056de8..2510a614e8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -153,9 +153,9 @@ object SparkBuild extends Build { "org.ow2.asm" % "asm" % "4.0", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", - "com.typesafe.akka" % "akka-actor" % "2.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-remote" % "2.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-slf4j" % "2.0.3" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", @@ -235,7 +235,7 @@ object SparkBuild extends Build { "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-zeromq" % "2.0.3" excludeAll(excludeNetty) + "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty) ) ) ++ assemblySettings ++ extraAssemblySettings From f347cc3f659d4414a21de26feadcbe23a130e622 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 10:53:30 -0700 Subject: [PATCH 079/221] Fix deprecation warning and style issues --- core/src/test/scala/spark/FileServerSuite.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index 9c24ca430d..c7855a7bd3 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -35,26 +35,26 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal - }.collect + }.collect() assert(result.toSet === Set((1,200), (2,300), (3,500))) } test("Distributing files locally using URL as input") { // addFile("file:///....") sc = new SparkContext("local[4]", "test") - sc.addFile((new File(tmpFile.toString)).toURL.toString) + sc.addFile(new File(tmpFile.toString).toURI.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal - }.collect + }.collect() assert(result.toSet === Set((1,200), (2,300), (3,500))) } @@ -80,11 +80,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal - }.collect + }.collect() assert(result.toSet === Set((1,200), (2,300), (3,500))) } From 5c388808a81ebfb1fc23511882c18f9ae76ec509 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 11:01:14 -0700 Subject: [PATCH 080/221] SPARK-814: Result stages should be named after action --- .../scala/spark/scheduler/DAGScheduler.scala | 22 ++++++++++++------- .../main/scala/spark/scheduler/Stage.scala | 5 +++-- .../main/scala/spark/ui/jobs/IndexPage.scala | 2 +- .../spark/scheduler/JobLoggerSuite.scala | 4 ++-- 4 files changed, 20 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 3d3b9ea011..30a648f50b 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -150,7 +150,13 @@ class DAGScheduler( * as a result stage for the final RDD used directly in an action. The stage will also be given * the provided priority. */ - private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = { + private def newStage( + rdd: RDD[_], + shuffleDep: Option[ShuffleDependency[_,_]], + priority: Int, + callSite: Option[String] = None) + : Stage = + { if (shuffleDep != None) { // Kind of ugly: need to register RDDs with the cache and map output tracker here // since we can't do it in the RDD constructor because # of partitions is unknown @@ -158,7 +164,7 @@ class DAGScheduler( mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size) } val id = nextStageId.getAndIncrement() - val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) + val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority, callSite) idToStage(id) = stage stageToInfos(stage) = StageInfo(stage) stage @@ -286,12 +292,12 @@ class DAGScheduler( event match { case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) => val runId = nextRunId.getAndIncrement() - val finalStage = newStage(finalRDD, None, runId) + val finalStage = newStage(finalRDD, None, runId, Some(callSite)) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") - logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") + 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) { @@ -502,7 +508,7 @@ class DAGScheduler( case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0) case _ => "Unkown" } - logInfo("%s (%s) finished in %s s".format(stage, stage.origin, serviceTime)) + logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stage.completionTime = Some(System.currentTimeMillis) val stageComp = StageCompleted(stageToInfos(stage)) sparkListeners.foreach{_.onStageCompleted(stageComp)} @@ -568,7 +574,7 @@ class DAGScheduler( if (stage.outputLocs.count(_ == Nil) != 0) { // Some tasks had failed; let's resubmit this stage // TODO: Lower-level scheduler should also deal with this - logInfo("Resubmitting " + stage + " (" + stage.origin + + logInfo("Resubmitting " + stage + " (" + stage.name + ") because some of its tasks had failed: " + stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) submitStage(stage) @@ -600,7 +606,7 @@ class DAGScheduler( running -= failedStage failed += failedStage // TODO: Cancel running tasks in the stage - logInfo("Marking " + failedStage + " (" + failedStage.origin + + logInfo("Marking " + failedStage + " (" + failedStage.name + ") for resubmision due to a fetch failure") // Mark the map whose fetch failed as broken in the map stage val mapStage = shuffleToMapStage(shuffleId) @@ -608,7 +614,7 @@ class DAGScheduler( mapStage.removeOutputLoc(mapId, bmAddress) mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) } - logInfo("The failed fetch was from " + mapStage + " (" + mapStage.origin + + logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + "); marking it for resubmission") failed += mapStage // Remember that a fetch failed now; this is used to resubmit the broken diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 539cf8233b..4937eb3b88 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -24,7 +24,8 @@ private[spark] class Stage( val rdd: RDD[_], val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage val parents: List[Stage], - val priority: Int) + val priority: Int, + callSite: Option[String]) extends Logging { val isShuffleMap = shuffleDep != None @@ -85,7 +86,7 @@ private[spark] class Stage( return id } - def origin: String = rdd.origin + val name = callSite.getOrElse(rdd.origin) override def toString = "Stage " + id diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 7907ab3bc7..2df5f0192b 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -89,7 +89,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { {s.id} - {s.origin} + {s.name} {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 699901f1a1..2474d744b8 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -37,8 +37,8 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val parentRdd = makeRdd(4, Nil) val shuffleDep = new ShuffleDependency(parentRdd, null) val rootRdd = makeRdd(4, List(shuffleDep)) - val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) - val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) + val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID, None) + val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID, None) joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4)) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) From 00a14deb6d9b6d89e0ddbe1540636622b1dc4d16 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 11:52:20 -0700 Subject: [PATCH 081/221] Update to latest Scala Maven plugin and allow Zinc external compiler --- pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 63dd80b5fe..6fa1f1d4f1 100644 --- a/pom.xml +++ b/pom.xml @@ -319,7 +319,7 @@ net.alchim31.maven scala-maven-plugin - 3.1.0 + 3.1.5 scala-compile-first @@ -346,12 +346,14 @@ ${scala.version} incremental + true -unchecked -optimise -Xms64m + -Xms1024m -Xmx1024m -XX:PermSize=${PermGen} -XX:MaxPermSize=${MaxPermGen} From af3c9d50424602f3e5af1055e83e9badef0a1632 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 17:21:33 -0700 Subject: [PATCH 082/221] Add Apache license headers and LICENSE and NOTICE files --- LICENSE | 225 ++++++++++++++++-- NOTICE | 5 + bagel/pom.xml | 17 ++ bagel/src/main/scala/spark/bagel/Bagel.scala | 17 ++ .../spark/bagel/examples/PageRankUtils.scala | 17 ++ .../bagel/examples/WikipediaPageRank.scala | 17 ++ .../WikipediaPageRankStandalone.scala | 17 ++ bagel/src/test/resources/log4j.properties | 19 +- bagel/src/test/scala/bagel/BagelSuite.scala | 17 ++ bin/slaves.sh | 5 +- bin/spark-config.sh | 19 +- bin/spark-daemon.sh | 5 +- bin/spark-daemons.sh | 17 ++ bin/start-all.sh | 17 ++ bin/start-master.sh | 17 ++ bin/start-slave.sh | 17 ++ bin/start-slaves.sh | 17 ++ bin/stop-all.sh | 17 ++ bin/stop-master.sh | 17 ++ bin/stop-slaves.sh | 17 ++ core/pom.xml | 17 ++ .../hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 17 ++ .../yarn/ApplicationMasterArguments.scala | 17 ++ .../scala/spark/deploy/yarn/Client.scala | 17 ++ .../spark/deploy/yarn/ClientArguments.scala | 17 ++ .../spark/deploy/yarn/WorkerRunnable.scala | 17 ++ .../deploy/yarn/YarnAllocationHandler.scala | 17 ++ .../cluster/YarnClusterScheduler.scala | 17 ++ .../hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../java/spark/network/netty/FileClient.java | 17 ++ .../netty/FileClientChannelInitializer.java | 17 ++ .../network/netty/FileClientHandler.java | 17 ++ .../java/spark/network/netty/FileServer.java | 17 ++ .../netty/FileServerChannelInitializer.java | 17 ++ .../network/netty/FileServerHandler.java | 17 ++ .../spark/network/netty/PathResolver.java | 17 ++ core/src/main/scala/spark/Accumulators.scala | 17 ++ core/src/main/scala/spark/Aggregator.scala | 17 ++ .../spark/BlockStoreShuffleFetcher.scala | 17 ++ core/src/main/scala/spark/Cache.scala | 17 ++ core/src/main/scala/spark/CacheManager.scala | 17 ++ .../src/main/scala/spark/ClosureCleaner.scala | 17 ++ core/src/main/scala/spark/Dependency.scala | 17 ++ .../main/scala/spark/DoubleRDDFunctions.scala | 17 ++ .../scala/spark/FetchFailedException.scala | 17 ++ core/src/main/scala/spark/HadoopWriter.scala | 17 ++ .../src/main/scala/spark/HttpFileServer.scala | 19 +- core/src/main/scala/spark/HttpServer.scala | 17 ++ .../src/main/scala/spark/JavaSerializer.scala | 17 ++ .../src/main/scala/spark/KryoSerializer.scala | 17 ++ core/src/main/scala/spark/Logging.scala | 17 ++ .../main/scala/spark/MapOutputTracker.scala | 17 ++ .../main/scala/spark/PairRDDFunctions.scala | 17 ++ core/src/main/scala/spark/Partition.scala | 17 ++ core/src/main/scala/spark/Partitioner.scala | 17 ++ core/src/main/scala/spark/RDD.scala | 17 ++ .../main/scala/spark/RDDCheckpointData.scala | 17 ++ .../spark/SequenceFileRDDFunctions.scala | 17 ++ .../scala/spark/SerializableWritable.scala | 17 ++ .../src/main/scala/spark/ShuffleFetcher.scala | 17 ++ core/src/main/scala/spark/SizeEstimator.scala | 17 ++ .../main/scala/spark/SoftReferenceCache.scala | 17 ++ core/src/main/scala/spark/SparkContext.scala | 17 ++ core/src/main/scala/spark/SparkEnv.scala | 17 ++ .../src/main/scala/spark/SparkException.scala | 17 ++ core/src/main/scala/spark/SparkFiles.java | 17 ++ core/src/main/scala/spark/TaskContext.scala | 17 ++ core/src/main/scala/spark/TaskEndReason.scala | 17 ++ core/src/main/scala/spark/TaskState.scala | 17 ++ core/src/main/scala/spark/Utils.scala | 17 ++ .../scala/spark/api/java/JavaDoubleRDD.scala | 17 ++ .../scala/spark/api/java/JavaPairRDD.scala | 17 ++ .../main/scala/spark/api/java/JavaRDD.scala | 17 ++ .../scala/spark/api/java/JavaRDDLike.scala | 17 ++ .../spark/api/java/JavaSparkContext.scala | 17 ++ .../JavaSparkContextVarargsWorkaround.java | 17 ++ .../scala/spark/api/java/StorageLevels.java | 17 ++ .../java/function/DoubleFlatMapFunction.java | 17 ++ .../api/java/function/DoubleFunction.java | 17 ++ .../api/java/function/FlatMapFunction.scala | 17 ++ .../api/java/function/FlatMapFunction2.scala | 17 ++ .../spark/api/java/function/Function.java | 17 ++ .../spark/api/java/function/Function2.java | 17 ++ .../java/function/PairFlatMapFunction.java | 17 ++ .../spark/api/java/function/PairFunction.java | 17 ++ .../api/java/function/VoidFunction.scala | 19 +- .../api/java/function/WrappedFunction1.scala | 17 ++ .../api/java/function/WrappedFunction2.scala | 17 ++ .../spark/api/python/PythonPartitioner.scala | 17 ++ .../scala/spark/api/python/PythonRDD.scala | 17 ++ .../api/python/PythonWorkerFactory.scala | 17 ++ .../spark/broadcast/BitTorrentBroadcast.scala | 17 ++ .../scala/spark/broadcast/Broadcast.scala | 17 ++ .../spark/broadcast/BroadcastFactory.scala | 17 ++ .../scala/spark/broadcast/HttpBroadcast.scala | 17 ++ .../scala/spark/broadcast/MultiTracker.scala | 17 ++ .../scala/spark/broadcast/SourceInfo.scala | 17 ++ .../scala/spark/broadcast/TreeBroadcast.scala | 17 ++ .../spark/deploy/ApplicationDescription.scala | 17 ++ .../src/main/scala/spark/deploy/Command.scala | 17 ++ .../scala/spark/deploy/DeployMessage.scala | 17 ++ .../scala/spark/deploy/ExecutorState.scala | 17 ++ .../scala/spark/deploy/JsonProtocol.scala | 19 +- .../spark/deploy/LocalSparkCluster.scala | 17 ++ core/src/main/scala/spark/deploy/WebUI.scala | 17 ++ .../scala/spark/deploy/client/Client.scala | 17 ++ .../spark/deploy/client/ClientListener.scala | 17 ++ .../spark/deploy/client/TestClient.scala | 17 ++ .../spark/deploy/client/TestExecutor.scala | 17 ++ .../spark/deploy/master/ApplicationInfo.scala | 17 ++ .../deploy/master/ApplicationState.scala | 17 ++ .../spark/deploy/master/ExecutorInfo.scala | 17 ++ .../scala/spark/deploy/master/Master.scala | 17 ++ .../spark/deploy/master/MasterArguments.scala | 17 ++ .../spark/deploy/master/WorkerInfo.scala | 17 ++ .../spark/deploy/master/WorkerState.scala | 17 ++ .../deploy/master/ui/ApplicationPage.scala | 17 ++ .../spark/deploy/master/ui/IndexPage.scala | 17 ++ .../spark/deploy/master/ui/MasterWebUI.scala | 19 +- .../spark/deploy/worker/ExecutorRunner.scala | 17 ++ .../scala/spark/deploy/worker/Worker.scala | 17 ++ .../spark/deploy/worker/WorkerArguments.scala | 17 ++ .../spark/deploy/worker/ui/IndexPage.scala | 17 ++ .../spark/deploy/worker/ui/WorkerWebUI.scala | 17 ++ .../main/scala/spark/executor/Executor.scala | 17 ++ .../spark/executor/ExecutorBackend.scala | 17 ++ .../spark/executor/ExecutorExitCode.scala | 17 ++ .../executor/ExecutorURLClassLoader.scala | 17 ++ .../spark/executor/MesosExecutorBackend.scala | 17 ++ .../executor/StandaloneExecutorBackend.scala | 17 ++ .../scala/spark/executor/TaskMetrics.scala | 17 ++ .../scala/spark/network/BufferMessage.scala | 19 +- .../main/scala/spark/network/Connection.scala | 17 ++ .../spark/network/ConnectionManager.scala | 17 ++ .../spark/network/ConnectionManagerId.scala | 17 ++ .../spark/network/ConnectionManagerTest.scala | 17 ++ .../main/scala/spark/network/Message.scala | 17 ++ .../scala/spark/network/MessageChunk.scala | 17 ++ .../spark/network/MessageChunkHeader.scala | 17 ++ .../scala/spark/network/ReceiverTest.scala | 17 ++ .../main/scala/spark/network/SenderTest.scala | 17 ++ .../spark/network/netty/FileHeader.scala | 17 ++ .../spark/network/netty/ShuffleCopier.scala | 17 ++ .../spark/network/netty/ShuffleSender.scala | 17 ++ core/src/main/scala/spark/package.scala | 17 ++ .../partial/ApproximateActionListener.scala | 17 ++ .../spark/partial/ApproximateEvaluator.scala | 17 ++ .../scala/spark/partial/BoundedDouble.scala | 17 ++ .../scala/spark/partial/CountEvaluator.scala | 17 ++ .../spark/partial/GroupedCountEvaluator.scala | 17 ++ .../spark/partial/GroupedMeanEvaluator.scala | 17 ++ .../spark/partial/GroupedSumEvaluator.scala | 17 ++ .../scala/spark/partial/MeanEvaluator.scala | 17 ++ .../scala/spark/partial/PartialResult.scala | 17 ++ .../scala/spark/partial/StudentTCacher.scala | 17 ++ .../scala/spark/partial/SumEvaluator.scala | 17 ++ core/src/main/scala/spark/rdd/BlockRDD.scala | 17 ++ .../main/scala/spark/rdd/CartesianRDD.scala | 17 ++ .../main/scala/spark/rdd/CheckpointRDD.scala | 17 ++ .../main/scala/spark/rdd/CoGroupedRDD.scala | 17 ++ .../main/scala/spark/rdd/CoalescedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/EmptyRDD.scala | 17 ++ .../main/scala/spark/rdd/FilteredRDD.scala | 17 ++ .../main/scala/spark/rdd/FlatMappedRDD.scala | 17 ++ .../src/main/scala/spark/rdd/GlommedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/HadoopRDD.scala | 17 ++ core/src/main/scala/spark/rdd/JdbcRDD.scala | 17 ++ .../scala/spark/rdd/MapPartitionsRDD.scala | 17 ++ .../spark/rdd/MapPartitionsWithIndexRDD.scala | 17 ++ core/src/main/scala/spark/rdd/MappedRDD.scala | 17 ++ .../main/scala/spark/rdd/NewHadoopRDD.scala | 17 ++ .../spark/rdd/ParallelCollectionRDD.scala | 17 ++ .../scala/spark/rdd/PartitionPruningRDD.scala | 17 ++ core/src/main/scala/spark/rdd/PipedRDD.scala | 17 ++ .../src/main/scala/spark/rdd/SampledRDD.scala | 17 ++ .../main/scala/spark/rdd/ShuffledRDD.scala | 17 ++ .../main/scala/spark/rdd/SubtractedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/UnionRDD.scala | 17 ++ .../scala/spark/rdd/ZippedPartitionsRDD.scala | 17 ++ core/src/main/scala/spark/rdd/ZippedRDD.scala | 17 ++ .../scala/spark/scheduler/ActiveJob.scala | 17 ++ .../scala/spark/scheduler/DAGScheduler.scala | 17 ++ .../spark/scheduler/DAGSchedulerEvent.scala | 17 ++ .../spark/scheduler/InputFormatInfo.scala | 17 ++ .../scala/spark/scheduler/JobListener.scala | 17 ++ .../scala/spark/scheduler/JobLogger.scala | 17 ++ .../scala/spark/scheduler/JobResult.scala | 17 ++ .../scala/spark/scheduler/JobWaiter.scala | 17 ++ .../scala/spark/scheduler/MapStatus.scala | 17 ++ .../scala/spark/scheduler/ResultTask.scala | 17 ++ .../spark/scheduler/ShuffleMapTask.scala | 17 ++ .../scala/spark/scheduler/SparkListener.scala | 17 ++ .../scala/spark/scheduler/SplitInfo.scala | 17 ++ .../main/scala/spark/scheduler/Stage.scala | 17 ++ .../scala/spark/scheduler/StageInfo.scala | 19 +- .../src/main/scala/spark/scheduler/Task.scala | 17 ++ .../scala/spark/scheduler/TaskResult.scala | 17 ++ .../scala/spark/scheduler/TaskScheduler.scala | 17 ++ .../scheduler/TaskSchedulerListener.scala | 17 ++ .../main/scala/spark/scheduler/TaskSet.scala | 17 ++ .../scheduler/cluster/ClusterScheduler.scala | 17 ++ .../cluster/ClusterTaskSetManager.scala | 17 ++ .../cluster/ExecutorLossReason.scala | 17 ++ .../scala/spark/scheduler/cluster/Pool.scala | 17 ++ .../spark/scheduler/cluster/Schedulable.scala | 17 ++ .../cluster/SchedulableBuilder.scala | 17 ++ .../scheduler/cluster/SchedulerBackend.scala | 17 ++ .../cluster/SchedulingAlgorithm.scala | 17 ++ .../scheduler/cluster/SchedulingMode.scala | 17 ++ .../cluster/SparkDeploySchedulerBackend.scala | 17 ++ .../cluster/StandaloneClusterMessage.scala | 17 ++ .../cluster/StandaloneSchedulerBackend.scala | 17 ++ .../scheduler/cluster/TaskDescription.scala | 17 ++ .../spark/scheduler/cluster/TaskInfo.scala | 17 ++ .../scheduler/cluster/TaskSetManager.scala | 17 ++ .../spark/scheduler/cluster/WorkerOffer.scala | 17 ++ .../scheduler/local/LocalScheduler.scala | 17 ++ .../scheduler/local/LocalTaskSetManager.scala | 17 ++ .../mesos/CoarseMesosSchedulerBackend.scala | 17 ++ .../mesos/MesosSchedulerBackend.scala | 17 ++ .../scala/spark/serializer/Serializer.scala | 17 ++ .../spark/serializer/SerializerManager.scala | 17 ++ .../scala/spark/storage/BlockException.scala | 17 ++ .../spark/storage/BlockFetchTracker.scala | 17 ++ .../spark/storage/BlockFetcherIterator.scala | 17 ++ .../scala/spark/storage/BlockManager.scala | 17 ++ .../scala/spark/storage/BlockManagerId.scala | 17 ++ .../spark/storage/BlockManagerMaster.scala | 17 ++ .../storage/BlockManagerMasterActor.scala | 17 ++ .../spark/storage/BlockManagerMessages.scala | 17 ++ .../storage/BlockManagerSlaveActor.scala | 17 ++ .../spark/storage/BlockManagerWorker.scala | 17 ++ .../scala/spark/storage/BlockMessage.scala | 17 ++ .../spark/storage/BlockMessageArray.scala | 17 ++ .../spark/storage/BlockObjectWriter.scala | 17 ++ .../main/scala/spark/storage/BlockStore.scala | 17 ++ .../main/scala/spark/storage/DiskStore.scala | 17 ++ .../scala/spark/storage/MemoryStore.scala | 17 ++ .../main/scala/spark/storage/PutResult.scala | 17 ++ .../spark/storage/ShuffleBlockManager.scala | 17 ++ .../scala/spark/storage/StorageLevel.scala | 17 ++ .../scala/spark/storage/StorageUtils.scala | 17 ++ .../scala/spark/storage/ThreadingTest.scala | 17 ++ core/src/main/scala/spark/ui/JettyUtils.scala | 17 ++ core/src/main/scala/spark/ui/Page.scala | 19 +- core/src/main/scala/spark/ui/SparkUI.scala | 17 ++ core/src/main/scala/spark/ui/UIUtils.scala | 19 +- .../scala/spark/ui/UIWorkloadGenerator.scala | 17 ++ .../scala/spark/ui/env/EnvironmentUI.scala | 17 ++ .../main/scala/spark/ui/jobs/IndexPage.scala | 17 ++ .../scala/spark/ui/jobs/JobProgressUI.scala | 19 +- .../main/scala/spark/ui/jobs/StagePage.scala | 17 ++ .../spark/ui/storage/BlockManagerUI.scala | 17 ++ .../scala/spark/ui/storage/IndexPage.scala | 17 ++ .../main/scala/spark/ui/storage/RDDPage.scala | 17 ++ .../src/main/scala/spark/util/AkkaUtils.scala | 17 ++ .../spark/util/BoundedPriorityQueue.scala | 17 ++ .../spark/util/ByteBufferInputStream.scala | 17 ++ .../scala/spark/util/CompletionIterator.scala | 19 +- .../main/scala/spark/util/Distribution.scala | 19 +- .../main/scala/spark/util/IdGenerator.scala | 17 ++ core/src/main/scala/spark/util/IntParam.scala | 17 ++ .../main/scala/spark/util/MemoryParam.scala | 17 ++ .../scala/spark/util/MetadataCleaner.scala | 17 ++ .../main/scala/spark/util/NextIterator.scala | 19 +- .../spark/util/RateLimitedOutputStream.scala | 17 ++ .../scala/spark/util/SerializableBuffer.scala | 17 ++ .../main/scala/spark/util/StatCounter.scala | 17 ++ .../scala/spark/util/TimeStampedHashMap.scala | 17 ++ .../scala/spark/util/TimeStampedHashSet.scala | 17 ++ core/src/main/scala/spark/util/Vector.scala | 17 ++ core/src/test/resources/fairscheduler.xml | 17 ++ core/src/test/resources/log4j.properties | 19 +- .../test/scala/spark/AccumulatorSuite.scala | 17 ++ .../src/test/scala/spark/BroadcastSuite.scala | 17 ++ .../test/scala/spark/CheckpointSuite.scala | 17 ++ .../scala/spark/ClosureCleanerSuite.scala | 17 ++ .../test/scala/spark/DistributedSuite.scala | 17 ++ core/src/test/scala/spark/DriverSuite.scala | 17 ++ core/src/test/scala/spark/FailureSuite.scala | 17 ++ .../test/scala/spark/FileServerSuite.scala | 17 ++ core/src/test/scala/spark/FileSuite.scala | 17 ++ core/src/test/scala/spark/JavaAPISuite.java | 17 ++ .../scala/spark/KryoSerializerSuite.scala | 17 ++ .../test/scala/spark/LocalSparkContext.scala | 17 ++ .../scala/spark/MapOutputTrackerSuite.scala | 17 ++ .../scala/spark/PairRDDFunctionsSuite.scala | 17 ++ .../test/scala/spark/PartitioningSuite.scala | 17 ++ core/src/test/scala/spark/PipedRDDSuite.scala | 17 ++ core/src/test/scala/spark/RDDSuite.scala | 17 ++ .../test/scala/spark/SharedSparkContext.scala | 17 ++ .../test/scala/spark/ShuffleNettySuite.scala | 17 ++ core/src/test/scala/spark/ShuffleSuite.scala | 17 ++ .../test/scala/spark/SizeEstimatorSuite.scala | 17 ++ core/src/test/scala/spark/SortingSuite.scala | 17 ++ .../src/test/scala/spark/ThreadingSuite.scala | 17 ++ .../src/test/scala/spark/UnpersistSuite.scala | 17 ++ core/src/test/scala/spark/UtilsSuite.scala | 17 ++ .../scala/spark/ZippedPartitionsSuite.scala | 17 ++ .../test/scala/spark/rdd/JdbcRDDSuite.scala | 17 ++ .../rdd/ParallelCollectionSplitSuite.scala | 17 ++ .../scheduler/ClusterSchedulerSuite.scala | 17 ++ .../spark/scheduler/DAGSchedulerSuite.scala | 17 ++ .../spark/scheduler/JobLoggerSuite.scala | 17 ++ .../spark/scheduler/LocalSchedulerSuite.scala | 17 ++ .../spark/scheduler/SparkListenerSuite.scala | 17 ++ .../spark/scheduler/TaskContextSuite.scala | 17 ++ .../spark/storage/BlockManagerSuite.scala | 17 ++ core/src/test/scala/spark/ui/UISuite.scala | 17 ++ .../scala/spark/util/DistributionSuite.scala | 17 ++ .../scala/spark/util/NextIteratorSuite.scala | 17 ++ .../util/RateLimitedOutputStreamSuite.scala | 17 ++ docs/_plugins/copy_api_dirs.rb | 17 ++ ec2/spark-ec2 | 2 + ec2/spark_ec2.py | 2 + examples/pom.xml | 17 ++ .../main/java/spark/examples/JavaHdfsLR.java | 17 ++ .../main/java/spark/examples/JavaKMeans.java | 17 ++ .../java/spark/examples/JavaLogQuery.java | 17 ++ .../main/java/spark/examples/JavaSparkPi.java | 17 ++ .../src/main/java/spark/examples/JavaTC.java | 17 ++ .../java/spark/examples/JavaWordCount.java | 17 ++ .../examples/JavaFlumeEventCount.java | 17 ++ .../examples/JavaNetworkWordCount.java | 17 ++ .../streaming/examples/JavaQueueStream.java | 17 ++ .../scala/spark/examples/BroadcastTest.scala | 17 ++ .../scala/spark/examples/CassandraTest.scala | 17 ++ .../examples/ExceptionHandlingTest.scala | 17 ++ .../scala/spark/examples/GroupByTest.scala | 17 ++ .../main/scala/spark/examples/HBaseTest.scala | 19 +- .../main/scala/spark/examples/HdfsTest.scala | 17 ++ .../main/scala/spark/examples/LocalALS.scala | 17 ++ .../scala/spark/examples/LocalFileLR.scala | 17 ++ .../scala/spark/examples/LocalKMeans.scala | 17 ++ .../main/scala/spark/examples/LocalLR.scala | 17 ++ .../main/scala/spark/examples/LocalPi.scala | 17 ++ .../main/scala/spark/examples/LogQuery.scala | 17 ++ .../spark/examples/MultiBroadcastTest.scala | 17 ++ .../examples/SimpleSkewedGroupByTest.scala | 17 ++ .../spark/examples/SkewedGroupByTest.scala | 17 ++ .../main/scala/spark/examples/SparkALS.scala | 17 ++ .../scala/spark/examples/SparkHdfsLR.scala | 17 ++ .../scala/spark/examples/SparkKMeans.scala | 17 ++ .../main/scala/spark/examples/SparkLR.scala | 17 ++ .../main/scala/spark/examples/SparkPi.scala | 17 ++ .../main/scala/spark/examples/SparkTC.scala | 17 ++ .../streaming/examples/ActorWordCount.scala | 17 ++ .../streaming/examples/FlumeEventCount.scala | 17 ++ .../streaming/examples/HdfsWordCount.scala | 17 ++ .../streaming/examples/KafkaWordCount.scala | 17 ++ .../streaming/examples/NetworkWordCount.scala | 17 ++ .../streaming/examples/QueueStream.scala | 17 ++ .../streaming/examples/RawNetworkGrep.scala | 17 ++ .../examples/StatefulNetworkWordCount.scala | 17 ++ .../examples/TwitterAlgebirdCMS.scala | 17 ++ .../examples/TwitterAlgebirdHLL.scala | 17 ++ .../examples/TwitterPopularTags.scala | 17 ++ .../streaming/examples/ZeroMQWordCount.scala | 17 ++ .../clickstream/PageViewGenerator.scala | 17 ++ .../examples/clickstream/PageViewStream.scala | 17 ++ make-distribution.sh | 19 ++ .../scala/spark/mllib/clustering/KMeans.scala | 17 ++ .../spark/mllib/clustering/KMeansModel.scala | 17 ++ .../spark/mllib/clustering/LocalKMeans.scala | 17 ++ .../spark/mllib/optimization/Gradient.scala | 17 ++ .../mllib/optimization/GradientDescent.scala | 17 ++ .../spark/mllib/optimization/Updater.scala | 17 ++ .../spark/mllib/recommendation/ALS.scala | 17 ++ .../MatrixFactorizationModel.scala | 17 ++ .../mllib/regression/LogisticRegression.scala | 17 ++ .../LogisticRegressionGenerator.scala | 17 ++ .../spark/mllib/regression/Regression.scala | 17 ++ .../mllib/regression/RidgeRegression.scala | 17 ++ .../regression/RidgeRegressionGenerator.scala | 17 ++ .../main/scala/spark/mllib/util/MLUtils.scala | 17 ++ mllib/src/test/resources/log4j.properties | 19 +- .../spark/mllib/clustering/KMeansSuite.scala | 17 ++ .../spark/mllib/recommendation/ALSSuite.scala | 17 ++ .../regression/LogisticRegressionSuite.scala | 17 ++ .../regression/RidgeRegressionSuite.scala | 17 ++ pom.xml | 17 ++ project/SparkBuild.scala | 16 ++ project/build.properties | 17 ++ pyspark | 17 ++ python/epydoc.conf | 17 ++ python/examples/als.py | 17 ++ python/examples/kmeans.py | 17 ++ python/examples/logistic_regression.py | 17 ++ python/examples/pi.py | 17 ++ python/examples/transitive_closure.py | 17 ++ python/examples/wordcount.py | 17 ++ python/pyspark/accumulators.py | 17 ++ python/pyspark/broadcast.py | 17 ++ python/pyspark/context.py | 17 ++ python/pyspark/daemon.py | 17 ++ python/pyspark/files.py | 17 ++ python/pyspark/java_gateway.py | 17 ++ python/pyspark/rdd.py | 17 ++ python/pyspark/serializers.py | 17 ++ python/pyspark/shell.py | 17 ++ python/pyspark/tests.py | 17 ++ python/pyspark/worker.py | 17 ++ python/run-tests | 20 +- repl-bin/pom.xml | 17 ++ repl-bin/src/deb/bin/run | 19 +- repl-bin/src/deb/bin/spark-executor | 17 ++ repl-bin/src/deb/bin/spark-shell | 17 ++ repl/pom.xml | 17 ++ .../spark/repl/ExecutorClassLoader.scala | 17 ++ repl/src/main/scala/spark/repl/Main.scala | 17 ++ repl/src/test/resources/log4j.properties | 19 +- .../src/test/scala/spark/repl/ReplSuite.scala | 17 ++ run | 17 ++ run.cmd | 18 ++ run2.cmd | 17 ++ sbt/sbt | 20 ++ sbt/sbt.cmd | 20 ++ spark-executor | 18 ++ spark-shell | 19 ++ spark-shell.cmd | 18 ++ streaming/pom.xml | 17 ++ .../scala/spark/streaming/Checkpoint.scala | 17 ++ .../main/scala/spark/streaming/DStream.scala | 17 ++ .../streaming/DStreamCheckpointData.scala | 17 ++ .../scala/spark/streaming/DStreamGraph.scala | 17 ++ .../main/scala/spark/streaming/Duration.scala | 17 ++ .../main/scala/spark/streaming/Interval.scala | 17 ++ .../src/main/scala/spark/streaming/Job.scala | 17 ++ .../scala/spark/streaming/JobManager.scala | 17 ++ .../spark/streaming/NetworkInputTracker.scala | 17 ++ .../streaming/PairDStreamFunctions.scala | 17 ++ .../scala/spark/streaming/Scheduler.scala | 17 ++ .../spark/streaming/StreamingContext.scala | 17 ++ .../src/main/scala/spark/streaming/Time.scala | 19 +- .../streaming/api/java/JavaDStream.scala | 19 +- .../streaming/api/java/JavaDStreamLike.scala | 19 +- .../streaming/api/java/JavaPairDStream.scala | 17 ++ .../api/java/JavaStreamingContext.scala | 17 ++ .../streaming/dstream/CoGroupedDStream.scala | 17 ++ .../dstream/ConstantInputDStream.scala | 19 +- .../streaming/dstream/FileInputDStream.scala | 17 ++ .../streaming/dstream/FilteredDStream.scala | 17 ++ .../dstream/FlatMapValuedDStream.scala | 17 ++ .../streaming/dstream/FlatMappedDStream.scala | 17 ++ .../streaming/dstream/FlumeInputDStream.scala | 17 ++ .../streaming/dstream/ForEachDStream.scala | 17 ++ .../streaming/dstream/GlommedDStream.scala | 17 ++ .../streaming/dstream/InputDStream.scala | 17 ++ .../streaming/dstream/KafkaInputDStream.scala | 17 ++ .../dstream/MapPartitionedDStream.scala | 17 ++ .../streaming/dstream/MapValuedDStream.scala | 17 ++ .../streaming/dstream/MappedDStream.scala | 17 ++ .../dstream/NetworkInputDStream.scala | 17 ++ .../dstream/PluggableInputDStream.scala | 17 ++ .../streaming/dstream/QueueInputDStream.scala | 17 ++ .../streaming/dstream/RawInputDStream.scala | 17 ++ .../dstream/ReducedWindowedDStream.scala | 17 ++ .../streaming/dstream/ShuffledDStream.scala | 17 ++ .../dstream/SocketInputDStream.scala | 17 ++ .../streaming/dstream/StateDStream.scala | 17 ++ .../dstream/TransformedDStream.scala | 17 ++ .../dstream/TwitterInputDStream.scala | 17 ++ .../streaming/dstream/UnionDStream.scala | 17 ++ .../streaming/dstream/WindowedDStream.scala | 17 ++ .../streaming/receivers/ActorReceiver.scala | 17 ++ .../streaming/receivers/ZeroMQReceiver.scala | 17 ++ .../scala/spark/streaming/util/Clock.scala | 17 ++ .../streaming/util/MasterFailureTest.scala | 17 ++ .../spark/streaming/util/RawTextHelper.scala | 17 ++ .../spark/streaming/util/RawTextSender.scala | 17 ++ .../spark/streaming/util/RecurringTimer.scala | 17 ++ .../java/spark/streaming/JavaAPISuite.java | 17 ++ .../java/spark/streaming/JavaTestUtils.scala | 19 +- streaming/src/test/resources/log4j.properties | 19 +- .../streaming/BasicOperationsSuite.scala | 17 ++ .../spark/streaming/CheckpointSuite.scala | 17 ++ .../scala/spark/streaming/FailureSuite.scala | 17 ++ .../spark/streaming/InputStreamsSuite.scala | 17 ++ .../scala/spark/streaming/TestSuiteBase.scala | 17 ++ .../streaming/WindowOperationsSuite.scala | 17 ++ 488 files changed, 8446 insertions(+), 57 deletions(-) create mode 100644 NOTICE diff --git a/LICENSE b/LICENSE index d17afa1fc6..d645695673 100644 --- a/LICENSE +++ b/LICENSE @@ -1,27 +1,202 @@ -Copyright (c) 2010, Regents of the University of California. -All rights reserved. -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions -are met: - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - * Neither the name of the University of California, Berkeley nor the - names of its contributors may be used to endorse or promote - products derived from this software without specific prior written - permission. + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED -TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/NOTICE b/NOTICE new file mode 100644 index 0000000000..7cbb114b2a --- /dev/null +++ b/NOTICE @@ -0,0 +1,5 @@ +Apache Spark +Copyright 2013 The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/bagel/pom.xml b/bagel/pom.xml index b83a0ef6c0..60bbc49e6c 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/spark/bagel/Bagel.scala index 5ecdd7d004..80c8d53d2b 100644 --- a/bagel/src/main/scala/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/spark/bagel/Bagel.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel import spark._ diff --git a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala index b97d786ed4..de65e27fe0 100644 --- a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala +++ b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel.examples import spark._ diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala index bc32663e0f..a0c5ac9c18 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala +++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel.examples import spark._ diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala index 9d9d80d809..3c54a85f42 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala +++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel.examples import spark._ diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 83d05cab2f..5cdcf35b23 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file bagel/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the file bagel/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala index a09c978068..ef2d57fbd0 100644 --- a/bagel/src/test/scala/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/bagel/BagelSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel import org.scalatest.{FunSuite, Assertions, BeforeAndAfter} diff --git a/bin/slaves.sh b/bin/slaves.sh index 030581e058..c8fb5ca473 100755 --- a/bin/slaves.sh +++ b/bin/slaves.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# This Spark deploy script is a modified version of the Apache Hadoop deploy -# script, available under the Apache 2 license: # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -10,13 +8,14 @@ # (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 +# 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. +# # Run a shell command on all slave hosts. # diff --git a/bin/spark-config.sh b/bin/spark-config.sh index d4b6558866..cd2c7b7b0d 100755 --- a/bin/spark-config.sh +++ b/bin/spark-config.sh @@ -1,3 +1,20 @@ +# +# 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. +# + # included in all the spark scripts with source command # should not be executable directly # also should not be passed any arguments, since we need original $* @@ -16,4 +33,4 @@ this="$config_bin/$script" export SPARK_PREFIX=`dirname "$this"`/.. export SPARK_HOME=${SPARK_PREFIX} -export SPARK_CONF_DIR="$SPARK_HOME/conf" \ No newline at end of file +export SPARK_CONF_DIR="$SPARK_HOME/conf" diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh index 8ee3ec481f..a5b88ca785 100755 --- a/bin/spark-daemon.sh +++ b/bin/spark-daemon.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# This Spark deploy script is a modified version of the Apache Hadoop deploy -# script, available under the Apache 2 license: # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -10,13 +8,14 @@ # (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 +# 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. +# # Runs a Spark command as a daemon. # diff --git a/bin/spark-daemons.sh b/bin/spark-daemons.sh index 0619097e4d..354eb905a1 100755 --- a/bin/spark-daemons.sh +++ b/bin/spark-daemons.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Run a Spark command on all slave hosts. usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..." diff --git a/bin/start-all.sh b/bin/start-all.sh index b9891ad2f6..0182f1ab24 100755 --- a/bin/start-all.sh +++ b/bin/start-all.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Start all spark daemons. # Starts the master on this node. # Starts a worker on each node specified in conf/slaves diff --git a/bin/start-master.sh b/bin/start-master.sh index 83a3e1f3dc..2288fb19d7 100755 --- a/bin/start-master.sh +++ b/bin/start-master.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Starts the master on the machine this script is executed on. bin=`dirname "$0"` diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 1082c09eb1..d6db16882d 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash + # +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Usage: start-slave.sh # where is like "spark://localhost:7077" diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh index 4e05224190..dad7c3df76 100755 --- a/bin/start-slaves.sh +++ b/bin/start-slaves.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + bin=`dirname "$0"` bin=`cd "$bin"; pwd` diff --git a/bin/stop-all.sh b/bin/stop-all.sh index d352f6f631..a043ac0095 100755 --- a/bin/stop-all.sh +++ b/bin/stop-all.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Start all spark daemons. # Run this on the master nde diff --git a/bin/stop-master.sh b/bin/stop-master.sh index 172ee5891d..31a610bf9d 100755 --- a/bin/stop-master.sh +++ b/bin/stop-master.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Starts the master on the machine this script is executed on. bin=`dirname "$0"` diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh index fbfc594472..8e056f23d4 100755 --- a/bin/stop-slaves.sh +++ b/bin/stop-slaves.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Starts the master on the machine this script is executed on. bin=`dirname "$0"` diff --git a/core/pom.xml b/core/pom.xml index dbb2da9a9c..6329b2fbd8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index f286f2cf9c..25386b2796 100644 --- a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapred trait HadoopMapRedUtil { diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 264d421d14..b1002e0cac 100644 --- a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapreduce import org.apache.hadoop.conf.Configuration diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index f1c86de4cc..df55be1254 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index 875c0a220b..0f972b7a0b 100644 --- a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -1,4 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapred import org.apache.hadoop.mapreduce.TaskType diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 8bc6fb6dea..1a7cdf4788 100644 --- a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapreduce import org.apache.hadoop.conf.Configuration diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala index 301a57fffa..6122fdced0 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import collection.mutable.HashMap diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 6a0617cc06..1b06169739 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import java.net.Socket diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala index 1b00208511..8de44b1f66 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import spark.util.IntParam diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala index 514c17f241..8bcbfc2735 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import java.net.{InetSocketAddress, URI} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala index 07e7edea36..67aff03781 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import spark.util.MemoryParam diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala index cc6f3344a1..f458f2f6a1 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import java.net.URI diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala index 61dd72a651..b0af8baf08 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import spark.{Logging, Utils} diff --git a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala index ed732d36bf..307d96111c 100644 --- a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark._ diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index a0652d7fc7..4b3d84670c 100644 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapred trait HadoopMapRedUtil { diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 7fdbe322fd..aa3b1ed3a5 100644 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapreduce import org.apache.hadoop.conf.Configuration diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index f1c86de4cc..df55be1254 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index a4bb4bc701..0625a6d502 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import io.netty.bootstrap.Bootstrap; diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java index af25baf641..05ad4b61d7 100644 --- a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import io.netty.buffer.BufType; diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java index 9fc9449827..e8cd9801f6 100644 --- a/core/src/main/java/spark/network/netty/FileClientHandler.java +++ b/core/src/main/java/spark/network/netty/FileClientHandler.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import io.netty.buffer.ByteBuf; diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index dd3a557ae5..9f009a61d5 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import java.net.InetSocketAddress; diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java index 8f1f5c65cd..50c57a81a3 100644 --- a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import io.netty.channel.ChannelInitializer; diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/spark/network/netty/FileServerHandler.java index a78eddb1b5..176ba8da49 100644 --- a/core/src/main/java/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/spark/network/netty/FileServerHandler.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import java.io.File; diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/spark/network/netty/PathResolver.java index 302411672c..f446c55b19 100755 --- a/core/src/main/java/spark/network/netty/PathResolver.java +++ b/core/src/main/java/spark/network/netty/PathResolver.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala index 57c6df35be..6ff92ce833 100644 --- a/core/src/main/scala/spark/Accumulators.scala +++ b/core/src/main/scala/spark/Accumulators.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/spark/Aggregator.scala index df8ce9c054..136b4da61e 100644 --- a/core/src/main/scala/spark/Aggregator.scala +++ b/core/src/main/scala/spark/Aggregator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 3239f4c385..8f6953b1f5 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala index 20d677a854..b0c83ce59d 100644 --- a/core/src/main/scala/spark/Cache.scala +++ b/core/src/main/scala/spark/Cache.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.util.concurrent.atomic.AtomicInteger diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala index f7a2b7e802..81314805a9 100644 --- a/core/src/main/scala/spark/CacheManager.scala +++ b/core/src/main/scala/spark/CacheManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.{ArrayBuffer, HashSet} diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala index d5e7132ff9..8b39241095 100644 --- a/core/src/main/scala/spark/ClosureCleaner.scala +++ b/core/src/main/scala/spark/ClosureCleaner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.lang.reflect.Field diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index 2af44aa383..d17e70a4fa 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark /** diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala index 178d31a73b..93ef097702 100644 --- a/core/src/main/scala/spark/DoubleRDDFunctions.scala +++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import spark.partial.BoundedDouble diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/spark/FetchFailedException.scala index 40b0193f19..a2dae6cae9 100644 --- a/core/src/main/scala/spark/FetchFailedException.scala +++ b/core/src/main/scala/spark/FetchFailedException.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import spark.storage.BlockManagerId diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala index 5e8396edb9..b1fe0075a3 100644 --- a/core/src/main/scala/spark/HadoopWriter.scala +++ b/core/src/main/scala/spark/HadoopWriter.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapred import org.apache.hadoop.fs.FileSystem diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/spark/HttpFileServer.scala index 00901d95e2..a13a7a2859 100644 --- a/core/src/main/scala/spark/HttpFileServer.scala +++ b/core/src/main/scala/spark/HttpFileServer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.{File} @@ -42,4 +59,4 @@ private[spark] class HttpFileServer extends Logging { return dir + "/" + file.getName } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/HttpServer.scala b/core/src/main/scala/spark/HttpServer.scala index 4e0507c080..c9dffbc631 100644 --- a/core/src/main/scala/spark/HttpServer.scala +++ b/core/src/main/scala/spark/HttpServer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.File diff --git a/core/src/main/scala/spark/JavaSerializer.scala b/core/src/main/scala/spark/JavaSerializer.scala index b04a27d073..04c5f44e6b 100644 --- a/core/src/main/scala/spark/JavaSerializer.scala +++ b/core/src/main/scala/spark/JavaSerializer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index c7dbcc6fbc..ee37da7948 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala index 0fc8c31463..79b0362830 100644 --- a/core/src/main/scala/spark/Logging.scala +++ b/core/src/main/scala/spark/Logging.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.slf4j.Logger diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index fde597ffd1..2c417e31db 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 8b313c645f..6b0cc2fbf1 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/Partition.scala b/core/src/main/scala/spark/Partition.scala index e384308ef6..2a4edcec98 100644 --- a/core/src/main/scala/spark/Partition.scala +++ b/core/src/main/scala/spark/Partition.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark /** diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 6f8cd17c88..660af70d52 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark /** diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 8aa77266bc..ca7cdd622a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.util.Random diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala index 57e0405fb4..b615f820eb 100644 --- a/core/src/main/scala/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/spark/RDDCheckpointData.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.apache.hadoop.fs.Path diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 2911f9036e..9f30b7f22f 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.EOFException diff --git a/core/src/main/scala/spark/SerializableWritable.scala b/core/src/main/scala/spark/SerializableWritable.scala index 8306fbf570..0236611ef9 100644 --- a/core/src/main/scala/spark/SerializableWritable.scala +++ b/core/src/main/scala/spark/SerializableWritable.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 9513a00126..dcced035e7 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import spark.executor.TaskMetrics diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala index f8a4c4e489..6cc57566d7 100644 --- a/core/src/main/scala/spark/SizeEstimator.scala +++ b/core/src/main/scala/spark/SizeEstimator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.lang.reflect.Field diff --git a/core/src/main/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala index 3dd0a4b1f9..f41a379582 100644 --- a/core/src/main/scala/spark/SoftReferenceCache.scala +++ b/core/src/main/scala/spark/SoftReferenceCache.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import com.google.common.collect.MapMaker diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 43e6af2351..46b9935cb7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index ec59b4f48f..f2bdc11bdb 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import collection.mutable diff --git a/core/src/main/scala/spark/SparkException.scala b/core/src/main/scala/spark/SparkException.scala index aa7a16d7dd..b7045eea63 100644 --- a/core/src/main/scala/spark/SparkException.scala +++ b/core/src/main/scala/spark/SparkException.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark class SparkException(message: String, cause: Throwable) diff --git a/core/src/main/scala/spark/SparkFiles.java b/core/src/main/scala/spark/SparkFiles.java index 566aec622c..f9b3f7965e 100644 --- a/core/src/main/scala/spark/SparkFiles.java +++ b/core/src/main/scala/spark/SparkFiles.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark; import java.io.File; diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index dd0609026a..b79f4ca813 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import executor.TaskMetrics diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index bb75ec208c..3ad665da34 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import spark.executor.TaskMetrics diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala index 78eb33a628..9df7d8277b 100644 --- a/core/src/main/scala/spark/TaskState.scala +++ b/core/src/main/scala/spark/TaskState.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.apache.mesos.Protos.{TaskState => MesosTaskState} diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index a36186bf8a..e6a96a5ec1 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 16692c0440..392556f261 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import spark.RDD diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 76051597b6..ccc511dc5f 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import java.util.{List => JList} diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index 626b499454..c0bf2cf568 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import spark._ diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 27f40ecdfd..21b5abf053 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import java.util.{List => JList, Comparator} diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index 5f18b1e15b..fe182e7ab6 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import java.util.{Map => JMap} diff --git a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java index 97344e73da..42b1de01b1 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java +++ b/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java; import java.util.Arrays; diff --git a/core/src/main/scala/spark/api/java/StorageLevels.java b/core/src/main/scala/spark/api/java/StorageLevels.java index 5e5845ac3a..f385636e83 100644 --- a/core/src/main/scala/spark/api/java/StorageLevels.java +++ b/core/src/main/scala/spark/api/java/StorageLevels.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java; import spark.storage.StorageLevel; diff --git a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java index 3a8192be3a..8bc88d757f 100644 --- a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java +++ b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; diff --git a/core/src/main/scala/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFunction.java index c6ef76d088..1aa1e5dae0 100644 --- a/core/src/main/scala/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/scala/spark/api/java/function/DoubleFunction.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala index e027cdacd3..9eb0cfe3f9 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function /** diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala index 6044043add..dda98710c2 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function /** diff --git a/core/src/main/scala/spark/api/java/function/Function.java b/core/src/main/scala/spark/api/java/function/Function.java index dae8295f21..2a2ea0aacf 100644 --- a/core/src/main/scala/spark/api/java/function/Function.java +++ b/core/src/main/scala/spark/api/java/function/Function.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; import scala.reflect.ClassManifest; diff --git a/core/src/main/scala/spark/api/java/function/Function2.java b/core/src/main/scala/spark/api/java/function/Function2.java index 69bf12c8c9..952d31ece4 100644 --- a/core/src/main/scala/spark/api/java/function/Function2.java +++ b/core/src/main/scala/spark/api/java/function/Function2.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; import scala.reflect.ClassManifest; diff --git a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java index b3cc4df6aa..4aad602da3 100644 --- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; import scala.Tuple2; diff --git a/core/src/main/scala/spark/api/java/function/PairFunction.java b/core/src/main/scala/spark/api/java/function/PairFunction.java index 9fc6df4b88..ccfe64ecf1 100644 --- a/core/src/main/scala/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFunction.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; import scala.Tuple2; diff --git a/core/src/main/scala/spark/api/java/function/VoidFunction.scala b/core/src/main/scala/spark/api/java/function/VoidFunction.scala index b0096cf2bf..f6fc0b0f7d 100644 --- a/core/src/main/scala/spark/api/java/function/VoidFunction.scala +++ b/core/src/main/scala/spark/api/java/function/VoidFunction.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function /** @@ -13,4 +30,4 @@ abstract class VoidFunction[T] extends Serializable { // return Unit), so it is implicitly converted to a Function1[T, Unit]: object VoidFunction { implicit def toFunction[T](f: VoidFunction[T]) : Function1[T, Unit] = ((x : T) => f.call(x)) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala index 923f5cdf4f..1758a38c4e 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala +++ b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function import scala.runtime.AbstractFunction1 diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala index 2c6e9b1571..b093567d2c 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala +++ b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function import scala.runtime.AbstractFunction2 diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala index d618c098c2..31a719fbff 100644 --- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.python import spark.Partitioner diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 31d8ea89d4..af10822dbd 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.python import java.io._ diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 85d1dfeac8..078ad45ce8 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.python import java.io.{DataInputStream, IOException} diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala index adcb2d2415..6f7d385379 100644 --- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.io._ diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala index 415bde5d67..aba56a60ca 100644 --- a/core/src/main/scala/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/spark/broadcast/Broadcast.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.io._ diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala index 5c6184c3c7..d33d95c7d9 100644 --- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast /** diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala index 7e30b8f7d2..c565876950 100644 --- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/spark/broadcast/MultiTracker.scala index 3fd77af73f..7855d44e9b 100644 --- a/core/src/main/scala/spark/broadcast/MultiTracker.scala +++ b/core/src/main/scala/spark/broadcast/MultiTracker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.io._ diff --git a/core/src/main/scala/spark/broadcast/SourceInfo.scala b/core/src/main/scala/spark/broadcast/SourceInfo.scala index c79bb93c38..b17ae63b5c 100644 --- a/core/src/main/scala/spark/broadcast/SourceInfo.scala +++ b/core/src/main/scala/spark/broadcast/SourceInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.util.BitSet diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala index c55c476117..ea1e9a12c1 100644 --- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.io._ diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index 02193c7008..a8b22fbef8 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy private[spark] class ApplicationDescription( diff --git a/core/src/main/scala/spark/deploy/Command.scala b/core/src/main/scala/spark/deploy/Command.scala index 577101e3c3..bad629e965 100644 --- a/core/src/main/scala/spark/deploy/Command.scala +++ b/core/src/main/scala/spark/deploy/Command.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import scala.collection.Map diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 51274acb1e..e1f8aff6f5 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import spark.deploy.ExecutorState.ExecutorState diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/spark/deploy/ExecutorState.scala index 5dc0c54552..08c9a3b725 100644 --- a/core/src/main/scala/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/spark/deploy/ExecutorState.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy private[spark] object ExecutorState diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 335aaeb769..64f89623e1 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import master.{ApplicationInfo, WorkerInfo} @@ -62,4 +79,4 @@ private[spark] object JsonProtocol { ("executors" -> obj.executors.toList.map(writeExecutorRunner)) ~ ("finishedexecutors" -> obj.finishedExecutors.toList.map(writeExecutorRunner)) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 939f26b6f4..6b8e9f27af 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} diff --git a/core/src/main/scala/spark/deploy/WebUI.scala b/core/src/main/scala/spark/deploy/WebUI.scala index 844c4142c7..8ea7792ef4 100644 --- a/core/src/main/scala/spark/deploy/WebUI.scala +++ b/core/src/main/scala/spark/deploy/WebUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import java.text.SimpleDateFormat diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 4af44f9c16..edefa0292d 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.client import spark.deploy._ diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala index e8c4083f9d..064024455e 100644 --- a/core/src/main/scala/spark/deploy/client/ClientListener.scala +++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.client /** diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index f195082808..4f4daa141a 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.client import spark.util.AkkaUtils diff --git a/core/src/main/scala/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/spark/deploy/client/TestExecutor.scala index 0e46db2272..8a22b6b89f 100644 --- a/core/src/main/scala/spark/deploy/client/TestExecutor.scala +++ b/core/src/main/scala/spark/deploy/client/TestExecutor.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.client private[spark] object TestExecutor { diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 785c16e2be..15ff919738 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import spark.deploy.ApplicationDescription diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/spark/deploy/master/ApplicationState.scala index 15016b388d..94f0ad8bae 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationState.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master private[spark] object ApplicationState diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala index 48e6055fb5..99b60f7d09 100644 --- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import spark.deploy.ExecutorState diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 4dd6c448a9..e5a7a87e2e 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import akka.actor._ diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala index 3d28ecabb4..d0ec3d5ea0 100644 --- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import spark.util.IntParam diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala index 0c08c5f417..4135cfeb28 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import akka.actor.ActorRef diff --git a/core/src/main/scala/spark/deploy/master/WorkerState.scala b/core/src/main/scala/spark/deploy/master/WorkerState.scala index 0bf35014c8..3e50b7748d 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerState.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 8553377d8f..32264af393 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master.ui import akka.dispatch.Await diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 5e3c5e064f..b05197c1b9 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master.ui import akka.dispatch.Await diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index 82bc6961e2..04b32c7968 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master.ui import akka.actor.ActorRef @@ -56,4 +73,4 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten private[spark] object MasterWebUI { val STATIC_RESOURCE_DIR = "spark/ui/static" val DEFAULT_PORT = "8080" -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 5d3d54c65e..8b51ff1c3a 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker import java.io._ diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index f20ea42d7f..0bd88ea253 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker import scala.collection.mutable.{ArrayBuffer, HashMap} diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala index 2b96611ee3..9fcd3260ca 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker import spark.util.IntParam diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index c515f2e238..7548a26c2e 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker.ui import akka.dispatch.Await diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index ccd55c1ce4..61d4cd6d99 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker.ui import akka.actor.ActorRef diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 8360547a74..2e81151882 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.io.{File, FileOutputStream} diff --git a/core/src/main/scala/spark/executor/ExecutorBackend.scala b/core/src/main/scala/spark/executor/ExecutorBackend.scala index e97e509700..33a6f8a824 100644 --- a/core/src/main/scala/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/ExecutorBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/spark/executor/ExecutorExitCode.scala index fd76029cb3..64b9fb88f8 100644 --- a/core/src/main/scala/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/spark/executor/ExecutorExitCode.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor /** diff --git a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala index 5beb4d049e..09d12fb65b 100644 --- a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.net.{URLClassLoader, URL} diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 10f3531df0..4961c42fad 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index ebe2ac68d8..f4003da732 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 1dc13754f9..3151627839 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor class TaskMetrics extends Serializable { diff --git a/core/src/main/scala/spark/network/BufferMessage.scala b/core/src/main/scala/spark/network/BufferMessage.scala index 7b0e489a6c..e566aeac13 100644 --- a/core/src/main/scala/spark/network/BufferMessage.scala +++ b/core/src/main/scala/spark/network/BufferMessage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer @@ -91,4 +108,4 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: "BufferMessage(id = " + id + ", size = " + size + ")" } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index 6e28f677a3..b66c00b58c 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import spark._ diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 624a094856..6c4e7dc03e 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import spark._ diff --git a/core/src/main/scala/spark/network/ConnectionManagerId.scala b/core/src/main/scala/spark/network/ConnectionManagerId.scala index b554e84251..9d5c518293 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerId.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.net.InetSocketAddress diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala index 533e4610f3..9e3827aaf5 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import spark._ diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala index d4f03610eb..a25457ea35 100644 --- a/core/src/main/scala/spark/network/Message.scala +++ b/core/src/main/scala/spark/network/Message.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/MessageChunk.scala b/core/src/main/scala/spark/network/MessageChunk.scala index aaf9204d0e..784db5ab62 100644 --- a/core/src/main/scala/spark/network/MessageChunk.scala +++ b/core/src/main/scala/spark/network/MessageChunk.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/MessageChunkHeader.scala b/core/src/main/scala/spark/network/MessageChunkHeader.scala index 3693d509d6..18d0cbcc14 100644 --- a/core/src/main/scala/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/spark/network/MessageChunkHeader.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.net.InetAddress diff --git a/core/src/main/scala/spark/network/ReceiverTest.scala b/core/src/main/scala/spark/network/ReceiverTest.scala index a174d5f403..2bbc736f40 100644 --- a/core/src/main/scala/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/spark/network/ReceiverTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/SenderTest.scala b/core/src/main/scala/spark/network/SenderTest.scala index a4ff69e4d2..542c54c36b 100644 --- a/core/src/main/scala/spark/network/SenderTest.scala +++ b/core/src/main/scala/spark/network/SenderTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/netty/FileHeader.scala b/core/src/main/scala/spark/network/netty/FileHeader.scala index aed4254234..bf46d32aa3 100644 --- a/core/src/main/scala/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/spark/network/netty/FileHeader.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty import io.netty.buffer._ diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index 8d5194a737..b01f6369f6 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty import java.util.concurrent.Executors diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala index d6fa4b1e80..cdf88b03a0 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty import java.io.File diff --git a/core/src/main/scala/spark/package.scala b/core/src/main/scala/spark/package.scala index 389ec4da3e..b244bfbf06 100644 --- a/core/src/main/scala/spark/package.scala +++ b/core/src/main/scala/spark/package.scala @@ -1,3 +1,20 @@ +/* + * 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 functionality. [[spark.SparkContext]] serves as the main entry point to Spark, while * [[spark.RDD]] is the data type representing a distributed collection, and provides most diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala index de2dce161a..691d939150 100644 --- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import spark._ diff --git a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala index 75713b2eaa..5eae144dfb 100644 --- a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala +++ b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial /** diff --git a/core/src/main/scala/spark/partial/BoundedDouble.scala b/core/src/main/scala/spark/partial/BoundedDouble.scala index 463c33d6e2..8bdbe6c012 100644 --- a/core/src/main/scala/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/spark/partial/BoundedDouble.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial /** diff --git a/core/src/main/scala/spark/partial/CountEvaluator.scala b/core/src/main/scala/spark/partial/CountEvaluator.scala index daf2c5170c..6aa92094eb 100644 --- a/core/src/main/scala/spark/partial/CountEvaluator.scala +++ b/core/src/main/scala/spark/partial/CountEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala index 01fbb8a11b..ebe2e5a1e3 100644 --- a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala index c622df5220..2dadbbd5fb 100644 --- a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala index 20fa55cff2..ae2b63f7cb 100644 --- a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/partial/MeanEvaluator.scala b/core/src/main/scala/spark/partial/MeanEvaluator.scala index 762c85400d..5ddcad7075 100644 --- a/core/src/main/scala/spark/partial/MeanEvaluator.scala +++ b/core/src/main/scala/spark/partial/MeanEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/PartialResult.scala b/core/src/main/scala/spark/partial/PartialResult.scala index 200ed4ea1e..922a9f9bc6 100644 --- a/core/src/main/scala/spark/partial/PartialResult.scala +++ b/core/src/main/scala/spark/partial/PartialResult.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial class PartialResult[R](initialVal: R, isFinal: Boolean) { diff --git a/core/src/main/scala/spark/partial/StudentTCacher.scala b/core/src/main/scala/spark/partial/StudentTCacher.scala index 443abba5cd..f3bb987d46 100644 --- a/core/src/main/scala/spark/partial/StudentTCacher.scala +++ b/core/src/main/scala/spark/partial/StudentTCacher.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/SumEvaluator.scala b/core/src/main/scala/spark/partial/SumEvaluator.scala index 58fb60f441..4083abef03 100644 --- a/core/src/main/scala/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/spark/partial/SumEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 719d4bf03e..0ebb722d73 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 38600b8be4..150e5bca29 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.io.{ObjectOutputStream, IOException} diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 377b1bdbe0..6794e0e201 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark._ diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 8966f9f86e..c540cd36eb 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.io.{ObjectOutputStream, IOException} diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 6d862c0c28..2b5bf18541 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/spark/rdd/EmptyRDD.scala index e4dd3a7fa7..d7d4db5d30 100644 --- a/core/src/main/scala/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/spark/rdd/EmptyRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala index c84ec39d21..783508cfd1 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{OneToOneDependency, RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala index 8ebc778925..ed75eac3ff 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala index e16c7ba881..1573f8a289 100644 --- a/core/src/main/scala/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 07c103503c..d0fdeb741e 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.io.EOFException diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index a50f407737..59132437d2 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.sql.{Connection, ResultSet} diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala index d283c5b2bb..af8f0a112f 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala index afb7504ba1..3b4e9518fd 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala index af07311b6d..8b411dd85d 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 901d01ef30..17fe805fd4 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.text.SimpleDateFormat diff --git a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala index 07585a88ce..16ba0c26f8 100644 --- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import scala.collection.immutable.NumericRange diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 41ff62dd22..191cfde565 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index c0baf43d43..2cefdc78b0 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.io.PrintWriter diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index 243673f151..574c9b141d 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.util.Random diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index c7d1926b83..0137f80953 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 8a9efc5da2..0402b9f250 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 2c52a67e22..2776826f18 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index b234428ab2..6a4fa13ad6 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index f728e93d24..b1c43b3195 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala index 105eaecb22..71cc94edb6 100644 --- a/core/src/main/scala/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.TaskContext diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 64ed91f5a0..29e879aa42 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import cluster.TaskInfo diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index acad915f13..506c87f65b 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala index 17d0ea4f80..65f8c3200e 100644 --- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.Logging diff --git a/core/src/main/scala/spark/scheduler/JobListener.scala b/core/src/main/scala/spark/scheduler/JobListener.scala index f46b9d551d..af108b8fec 100644 --- a/core/src/main/scala/spark/scheduler/JobListener.scala +++ b/core/src/main/scala/spark/scheduler/JobListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler /** diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 6a9d52f356..85b5ddd4a8 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.io.PrintWriter diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala index a0fdf391e6..a61b335152 100644 --- a/core/src/main/scala/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/spark/scheduler/JobResult.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler /** diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala index 6ff2e29434..69cd161c1f 100644 --- a/core/src/main/scala/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/spark/scheduler/MapStatus.scala index 203abb917b..2f6a68ee85 100644 --- a/core/src/main/scala/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/spark/scheduler/MapStatus.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.storage.BlockManagerId diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 83166bce22..361b1e6b91 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark._ diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 95647389c3..1c25605f75 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.io._ diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 8de3aa91a4..4fb1c5d42d 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/SplitInfo.scala b/core/src/main/scala/spark/scheduler/SplitInfo.scala index 6abfb7a1f7..4e3661ec5d 100644 --- a/core/src/main/scala/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/spark/scheduler/SplitInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 4937eb3b88..5428daeb94 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.net.URI diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 8d83ff10c4..c4026f995a 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.scheduler.cluster.TaskInfo @@ -9,4 +26,4 @@ case class StageInfo( val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]() ) { override def toString = stage.rdd.toString -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index a6462c6968..50768d43e0 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.serializer.SerializerInstance diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 6de0aa7adf..dc0621ea7b 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.io._ diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala index 7787b54762..5188308006 100644 --- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler /** diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index b75d3736cf..245e7ccb52 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/spark/scheduler/TaskSet.scala index e4b5fcaedb..dc3550dd0b 100644 --- a/core/src/main/scala/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/spark/scheduler/TaskSet.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 3a0c29b27f..7c10074dc7 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import java.lang.{Boolean => JBoolean} diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 327d6797ae..3d06520675 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} diff --git a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala index 8bf838209f..8825f2dd24 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.executor.ExecutorExitCode diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index 941ba7a3f1..83708f07e1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 2dd9c0564f..f557b142c4 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index 18cc15c2a5..95554023c0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import java.io.{File, FileInputStream, FileOutputStream} diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala index 8844057a5c..4431744ec3 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.{SparkContext, Utils} diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index f33310a34a..69e0ac2a6b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster /** diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 6e0c6793e0..4b3e3e50e1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster object SchedulingMode extends Enumeration("FAIR","FIFO"){ diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 170ede0f44..55d6c0a47e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.{Utils, Logging, SparkContext} diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index 3335294844..ac9e5ef94d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.TaskState.TaskState diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 004592a540..03a64e0192 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala index b41e951be9..761fdf6919 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index 718f26bfbd..a1ebd48b01 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.Utils diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index b4dd75d90f..07c3ddcc7e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala index c47824315c..06d1203f70 100644 --- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala +++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster /** diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index b000e328e6..1f73cb99a7 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.local import java.io.File diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index f12fec41d5..e662ad6709 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.local import java.io.File diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index f4a2994b6d..7bc6040544 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.mesos import com.google.protobuf.ByteString diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index e83368b98d..75b8268b55 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.mesos import com.google.protobuf.ByteString diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala index 2ad73b711d..dc94d42bb6 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/spark/serializer/Serializer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.serializer import java.io.{EOFException, InputStream, OutputStream} diff --git a/core/src/main/scala/spark/serializer/SerializerManager.scala b/core/src/main/scala/spark/serializer/SerializerManager.scala index 60b2aac797..b7b24705a2 100644 --- a/core/src/main/scala/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/spark/serializer/SerializerManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.serializer import java.util.concurrent.ConcurrentHashMap diff --git a/core/src/main/scala/spark/storage/BlockException.scala b/core/src/main/scala/spark/storage/BlockException.scala index f275d476df..8ebfaf3cbf 100644 --- a/core/src/main/scala/spark/storage/BlockException.scala +++ b/core/src/main/scala/spark/storage/BlockException.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage private[spark] diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala index 0718156b1b..265e554ad8 100644 --- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala +++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage private[spark] trait BlockFetchTracker { diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index bec876213e..1965c5bc19 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9b39d3aadf..e4ffa57ad2 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{InputStream, OutputStream} diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index 1e557d6148..b36a6176c0 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 58888b1ebb..3186f7c85b 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import akka.actor.ActorRef diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 0d4384ba1f..244000d952 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 0010726c8d..01de4ccb8f 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{Externalizable, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala index b264d1deb5..45cffad810 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import akka.actor.Actor diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala index 3057ade233..39064bce92 100644 --- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala index 30d7500e01..ab72dbb62b 100644 --- a/core/src/main/scala/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/spark/storage/BlockMessage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala index ee0c5ff9a2..b0229d6124 100644 --- a/core/src/main/scala/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala index 42e2b07d5c..01ed6e8c1f 100644 --- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockStore.scala b/core/src/main/scala/spark/storage/BlockStore.scala index 8188d3595e..c8db0022b0 100644 --- a/core/src/main/scala/spark/storage/BlockStore.scala +++ b/core/src/main/scala/spark/storage/BlockStore.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index da859eebcb..3495d653bd 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala index eba5ee507f..b5a86b85a7 100644 --- a/core/src/main/scala/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/spark/storage/MemoryStore.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.util.LinkedHashMap diff --git a/core/src/main/scala/spark/storage/PutResult.scala b/core/src/main/scala/spark/storage/PutResult.scala index 76f236057b..3a0974fe15 100644 --- a/core/src/main/scala/spark/storage/PutResult.scala +++ b/core/src/main/scala/spark/storage/PutResult.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 44638e0c2d..8a7a6f9ed3 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import spark.serializer.Serializer diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index cc0c354e7e..f52650988c 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index aca16f533a..2aeed4ea3c 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import spark.{Utils, SparkContext} diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index 5c406e68cb..b3ab1ff4b4 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import akka.actor._ diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index bc6f9c10d5..ca6088ad93 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import annotation.tailrec diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala index ed8f91842c..a31e750d06 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/spark/ui/Page.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui -private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } \ No newline at end of file +private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 874e5ba8ec..9396f22063 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index fa46e2487d..b1d11954dd 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import scala.xml.Node @@ -115,4 +132,4 @@ private[spark] object UIUtils { } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 8bbc6ce88e..a80e2d7002 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import scala.util.Random diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index 6b8b9f05bb..5ae7935ed4 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.env import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 2df5f0192b..f31af3cda6 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.jobs import java.util.Date diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 84730cc091..44dcf82d11 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.jobs import akka.util.Duration @@ -124,4 +141,4 @@ private[spark] class JobProgressListener extends SparkListener { } return false // No tasks have finished for this stage } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 51b82b6a8c..292966f23a 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.jobs import java.util.Date diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index 3d67888e2c..49ed069c75 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.storage import akka.util.Duration diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index d284134391..4e0360d19a 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.storage import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 428db6fa95..003be54ad8 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.storage import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 18fc9673bf..9233277bdb 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import akka.actor.{ActorSystem, ExtendedActorSystem} diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala index 4bc5db8bb7..0575497f5d 100644 --- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.io.Serializable diff --git a/core/src/main/scala/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/spark/util/ByteBufferInputStream.scala index d7e67497fe..47a28e2f76 100644 --- a/core/src/main/scala/spark/util/ByteBufferInputStream.scala +++ b/core/src/main/scala/spark/util/ByteBufferInputStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.io.InputStream diff --git a/core/src/main/scala/spark/util/CompletionIterator.scala b/core/src/main/scala/spark/util/CompletionIterator.scala index 8139183780..210450892b 100644 --- a/core/src/main/scala/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/spark/util/CompletionIterator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util /** @@ -22,4 +39,4 @@ object CompletionIterator { def completion() = completionFunction } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala index 24738b4307..5d4d7a6c50 100644 --- a/core/src/main/scala/spark/util/Distribution.scala +++ b/core/src/main/scala/spark/util/Distribution.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.io.PrintStream @@ -62,4 +79,4 @@ object Distribution { quantiles.foreach{q => out.print(q + "\t")} out.println } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/util/IdGenerator.scala b/core/src/main/scala/spark/util/IdGenerator.scala index b6e309fe1a..3422280559 100644 --- a/core/src/main/scala/spark/util/IdGenerator.scala +++ b/core/src/main/scala/spark/util/IdGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.util.concurrent.atomic.AtomicInteger diff --git a/core/src/main/scala/spark/util/IntParam.scala b/core/src/main/scala/spark/util/IntParam.scala index 0427646747..daf0d58fa2 100644 --- a/core/src/main/scala/spark/util/IntParam.scala +++ b/core/src/main/scala/spark/util/IntParam.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util /** diff --git a/core/src/main/scala/spark/util/MemoryParam.scala b/core/src/main/scala/spark/util/MemoryParam.scala index 3726738842..298562323a 100644 --- a/core/src/main/scala/spark/util/MemoryParam.scala +++ b/core/src/main/scala/spark/util/MemoryParam.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import spark.Utils diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index dafa906712..92909e0959 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors} diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/spark/util/NextIterator.scala index 48b5018ddd..22163ece8d 100644 --- a/core/src/main/scala/spark/util/NextIterator.scala +++ b/core/src/main/scala/spark/util/NextIterator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util /** Provides a basic/boilerplate Iterator implementation. */ @@ -68,4 +85,4 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { gotNext = false nextValue } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala index e3f00ea8c7..00f782bbe7 100644 --- a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala +++ b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import scala.annotation.tailrec diff --git a/core/src/main/scala/spark/util/SerializableBuffer.scala b/core/src/main/scala/spark/util/SerializableBuffer.scala index 09d588fe1c..7e6842628a 100644 --- a/core/src/main/scala/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/spark/util/SerializableBuffer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/spark/util/StatCounter.scala index 2b980340b7..76358d4151 100644 --- a/core/src/main/scala/spark/util/StatCounter.scala +++ b/core/src/main/scala/spark/util/StatCounter.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util /** diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index 92dfaa6e6f..cc7909194a 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.util.concurrent.ConcurrentHashMap diff --git a/core/src/main/scala/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/spark/util/TimeStampedHashSet.scala index 5f1cc93752..41e3fd8cba 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashSet.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import scala.collection.mutable.Set diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/spark/util/Vector.scala index 835822edb2..ed49386f18 100644 --- a/core/src/main/scala/spark/util/Vector.scala +++ b/core/src/main/scala/spark/util/Vector.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util class Vector(val elements: Array[Double]) extends Serializable { diff --git a/core/src/test/resources/fairscheduler.xml b/core/src/test/resources/fairscheduler.xml index 6e573b1883..996ffb1864 100644 --- a/core/src/test/resources/fairscheduler.xml +++ b/core/src/test/resources/fairscheduler.xml @@ -1,4 +1,21 @@ + + 2 diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index d05cf3dec1..f6fef03689 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file core/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the file core/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index ac8ae7d308..0af175f316 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/BroadcastSuite.scala b/core/src/test/scala/spark/BroadcastSuite.scala index 362a31fb0d..785721ece8 100644 --- a/core/src/test/scala/spark/BroadcastSuite.scala +++ b/core/src/test/scala/spark/BroadcastSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 28a7b21b92..a84c89e3c9 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/ClosureCleanerSuite.scala b/core/src/test/scala/spark/ClosureCleanerSuite.scala index b2d0dd4627..7d2831e19c 100644 --- a/core/src/test/scala/spark/ClosureCleanerSuite.scala +++ b/core/src/test/scala/spark/ClosureCleanerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.NotSerializableException diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 0024ede828..e11efe459c 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import network.ConnectionManagerId diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index 31c3dd75fb..ed16b9d8ef 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.File diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index 8c1445a465..6c847b8fef 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index c7855a7bd3..242ae971f8 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import com.google.common.io.Files diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index e61ff7793d..1e2c257c4b 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.{FileWriter, PrintWriter, File} diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index d306124fca..aaf03e683b 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark; import java.io.File; diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index 327e2ff848..c3323dcbb3 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala index bd184222ed..ddc212d290 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.Suite diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index 6e585e1c3a..ce6cec0451 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala index 682d2745bf..b102eaf4e6 100644 --- a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.ArrayBuffer diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index 99e433e3bd..b1e0b2b4d0 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index d263bb00e9..35c04710a3 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 7f7d4c8211..cbddf4e523 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.HashMap diff --git a/core/src/test/scala/spark/SharedSparkContext.scala b/core/src/test/scala/spark/SharedSparkContext.scala index 1da79f9824..70c24515be 100644 --- a/core/src/test/scala/spark/SharedSparkContext.scala +++ b/core/src/test/scala/spark/SharedSparkContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.Suite diff --git a/core/src/test/scala/spark/ShuffleNettySuite.scala b/core/src/test/scala/spark/ShuffleNettySuite.scala index bfaffa953e..6bad6c1d13 100644 --- a/core/src/test/scala/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/spark/ShuffleNettySuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.BeforeAndAfterAll diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 950218fa28..3a56c26bef 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.ArrayBuffer diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala index b5c8525f91..1ef812dfbd 100644 --- a/core/src/test/scala/spark/SizeEstimatorSuite.scala +++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala index f7bf207c68..b933c4aab8 100644 --- a/core/src/test/scala/spark/SortingSuite.scala +++ b/core/src/test/scala/spark/SortingSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/ThreadingSuite.scala b/core/src/test/scala/spark/ThreadingSuite.scala index ff315b6693..f2acd0bd3c 100644 --- a/core/src/test/scala/spark/ThreadingSuite.scala +++ b/core/src/test/scala/spark/ThreadingSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.util.concurrent.Semaphore diff --git a/core/src/test/scala/spark/UnpersistSuite.scala b/core/src/test/scala/spark/UnpersistSuite.scala index 94776e7572..93977d16f4 100644 --- a/core/src/test/scala/spark/UnpersistSuite.scala +++ b/core/src/test/scala/spark/UnpersistSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 1e1260f606..31c3b25c50 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import com.google.common.base.Charsets diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala index 96cb295f45..5e6d7b09d8 100644 --- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/spark/ZippedPartitionsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.immutable.NumericRange diff --git a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala index 6afb0fa9bc..dc8ca941c1 100644 --- a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.{ BeforeAndAfter, FunSuite } diff --git a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala index d27a2538e4..d1276d541f 100644 --- a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import scala.collection.immutable.NumericRange diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 8e1ad27e14..8f81d0b6ee 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 30e6fef950..f802b66cf1 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import scala.collection.mutable.{Map, HashMap} diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 2474d744b8..0f855c38da 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.util.Properties diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala index 8bd813fd14..14bb58731b 100644 --- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala index 48aa67c543..392d67d67b 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala index 647bcaf860..95a6eee2fc 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index b9d5f9668e..b719d65342 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index fc0c160720..56c1fed6ad 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import scala.util.{Failure, Success, Try} diff --git a/core/src/test/scala/spark/util/DistributionSuite.scala b/core/src/test/scala/spark/util/DistributionSuite.scala index cc6249b1dd..6578b55e82 100644 --- a/core/src/test/scala/spark/util/DistributionSuite.scala +++ b/core/src/test/scala/spark/util/DistributionSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/util/NextIteratorSuite.scala b/core/src/test/scala/spark/util/NextIteratorSuite.scala index ed5b36da73..fdbd43d941 100644 --- a/core/src/test/scala/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/spark/util/NextIteratorSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala index 794063fb6d..4c0044202f 100644 --- a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala +++ b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import org.scalatest.FunSuite diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index c10ae595de..45ef4bba82 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -1,3 +1,20 @@ +# +# 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. +# + require 'fileutils' include FileUtils diff --git a/ec2/spark-ec2 b/ec2/spark-ec2 index 2714f19ba3..454057aa0d 100755 --- a/ec2/spark-ec2 +++ b/ec2/spark-ec2 @@ -1,5 +1,6 @@ #!/bin/sh +# # 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 @@ -15,6 +16,7 @@ # 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. +# cd "`dirname $0`" PYTHONPATH="./third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" python ./spark_ec2.py $@ diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 7affe6fffc..2ec3c007fb 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -1,6 +1,7 @@ #!/usr/bin/env python # -*- coding: utf-8 -*- +# # 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 @@ -16,6 +17,7 @@ # 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. +# from __future__ import with_statement diff --git a/examples/pom.xml b/examples/pom.xml index 1976765c3d..7a8d08fade 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/spark/examples/JavaHdfsLR.java index 8b0a9b6808..9485e0cfa9 100644 --- a/examples/src/main/java/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/spark/examples/JavaHdfsLR.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import spark.api.java.JavaRDD; diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/spark/examples/JavaKMeans.java index 626034eb0d..2d34776177 100644 --- a/examples/src/main/java/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/spark/examples/JavaKMeans.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import scala.Tuple2; diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/spark/examples/JavaLogQuery.java index 6b22e7120c..d22684d980 100644 --- a/examples/src/main/java/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/spark/examples/JavaLogQuery.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import com.google.common.collect.Lists; diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/spark/examples/JavaSparkPi.java index a15a967de8..d5f42fbb38 100644 --- a/examples/src/main/java/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/spark/examples/JavaSparkPi.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import spark.api.java.JavaRDD; diff --git a/examples/src/main/java/spark/examples/JavaTC.java b/examples/src/main/java/spark/examples/JavaTC.java index b319bdab44..559d7f9e53 100644 --- a/examples/src/main/java/spark/examples/JavaTC.java +++ b/examples/src/main/java/spark/examples/JavaTC.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import scala.Tuple2; diff --git a/examples/src/main/java/spark/examples/JavaWordCount.java b/examples/src/main/java/spark/examples/JavaWordCount.java index 9d4c7a252d..1af370c1c3 100644 --- a/examples/src/main/java/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/spark/examples/JavaWordCount.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import scala.Tuple2; diff --git a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java index e24c6ddaa7..096a9ae219 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples; import spark.api.java.function.Function; diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java index 3e57580fd4..c54d3f3d59 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples; import com.google.common.collect.Lists; diff --git a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java index 15b82c8da1..1f4a991542 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples; import com.google.common.collect.Lists; diff --git a/examples/src/main/scala/spark/examples/BroadcastTest.scala b/examples/src/main/scala/spark/examples/BroadcastTest.scala index ba59be1687..911490cb6c 100644 --- a/examples/src/main/scala/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/BroadcastTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 0fe1833e83..104bfd5204 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import org.apache.hadoop.mapreduce.Job diff --git a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala index 21a90f2e5a..67ddaec8d2 100644 --- a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/GroupByTest.scala b/examples/src/main/scala/spark/examples/GroupByTest.scala index a6603653f1..5cee413615 100644 --- a/examples/src/main/scala/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/spark/examples/GroupByTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala index 6e910154d4..4dd6c243ac 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark._ @@ -32,4 +49,4 @@ object HBaseTest { System.exit(0) } -} \ No newline at end of file +} diff --git a/examples/src/main/scala/spark/examples/HdfsTest.scala b/examples/src/main/scala/spark/examples/HdfsTest.scala index dd61c467f7..23258336e2 100644 --- a/examples/src/main/scala/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/spark/examples/HdfsTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark._ diff --git a/examples/src/main/scala/spark/examples/LocalALS.scala b/examples/src/main/scala/spark/examples/LocalALS.scala index 2de810e062..7a449a9d72 100644 --- a/examples/src/main/scala/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/spark/examples/LocalALS.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import scala.math.sqrt diff --git a/examples/src/main/scala/spark/examples/LocalFileLR.scala b/examples/src/main/scala/spark/examples/LocalFileLR.scala index f958ef9f72..c1f8d32aa8 100644 --- a/examples/src/main/scala/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/spark/examples/LocalFileLR.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/spark/examples/LocalKMeans.scala index 4849f216fb..0a0bc6f476 100644 --- a/examples/src/main/scala/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/spark/examples/LocalKMeans.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/LocalLR.scala b/examples/src/main/scala/spark/examples/LocalLR.scala index cd73f553d6..ab99bf1fbe 100644 --- a/examples/src/main/scala/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/spark/examples/LocalLR.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/LocalPi.scala b/examples/src/main/scala/spark/examples/LocalPi.scala index 9457472f2d..ccd69695df 100644 --- a/examples/src/main/scala/spark/examples/LocalPi.scala +++ b/examples/src/main/scala/spark/examples/LocalPi.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import scala.math.random diff --git a/examples/src/main/scala/spark/examples/LogQuery.scala b/examples/src/main/scala/spark/examples/LogQuery.scala index 6497596d35..e815ececf7 100644 --- a/examples/src/main/scala/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/spark/examples/LogQuery.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala index a0aaf60918..d0b1cf06e5 100644 --- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala index 461b84a2c6..d197bbaf7c 100644 --- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala index 435675f9de..4641b82444 100644 --- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/SparkALS.scala b/examples/src/main/scala/spark/examples/SparkALS.scala index 8fb3b0fb2a..ba0dfd8f9b 100644 --- a/examples/src/main/scala/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/spark/examples/SparkALS.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import scala.math.sqrt diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala index 3d080a0257..ef6e09a8e8 100644 --- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/SparkKMeans.scala b/examples/src/main/scala/spark/examples/SparkKMeans.scala index 4161c59fea..38ed3b149a 100644 --- a/examples/src/main/scala/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/spark/examples/SparkKMeans.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/SparkLR.scala b/examples/src/main/scala/spark/examples/SparkLR.scala index 2f41aeb376..52a0d69744 100644 --- a/examples/src/main/scala/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/spark/examples/SparkLR.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala index f598d2ff9c..00560ac9d1 100644 --- a/examples/src/main/scala/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/spark/examples/SparkPi.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import scala.math.random diff --git a/examples/src/main/scala/spark/examples/SparkTC.scala b/examples/src/main/scala/spark/examples/SparkTC.scala index 911ae8f168..bf988a953b 100644 --- a/examples/src/main/scala/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/spark/examples/SparkTC.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark._ diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index 3b847fe603..f97174aeae 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import scala.collection.mutable.LinkedList diff --git a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala index 39c76fd98a..3ab4fc2c37 100644 --- a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.util.IntParam diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala index 9389f8a38d..f5baec242d 100644 --- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index 9202e65e09..4929703ba2 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import java.util.Properties diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala index 704540c2bf..150fb5eb9c 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala index f450e21040..da36c8c23c 100644 --- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.RDD diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala index 175281e095..7fb680bcc3 100644 --- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.util.IntParam diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index 51c3c9f9b4..33ab324732 100644 --- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming._ diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index 528778ed72..8770abd57e 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index 896e9fd8af..cba5c986be 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 65f0b6d352..682b99f75e 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala index 74d0d338b7..e264fae609 100644 --- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import akka.actor.ActorSystem diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala index 4c6e08bc74..375d5c9d22 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples.clickstream import java.net.{InetAddress,ServerSocket,Socket,SocketException} diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala index e226a4a73a..a24342bebf 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples.clickstream import spark.streaming.{Seconds, StreamingContext} diff --git a/make-distribution.sh b/make-distribution.sh index feb13d52f9..0116215163 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -1,4 +1,22 @@ #!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # # Script to create a binary distribution for easy deploys of Spark. # The distribution directory defaults to dist/ but can be overridden below. @@ -12,6 +30,7 @@ # 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. # 4) ./bin/start-slave.sh 1 <> # 5) MASTER="spark://my-master-ip:7077" ./spark-shell +# # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala index b0e141ff32..d875d6de50 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.clustering import scala.collection.mutable.ArrayBuffer diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala index 4fd0646160..b8f80e80cd 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.clustering import spark.RDD diff --git a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala index e12b3be251..89fe7d7e85 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.clustering import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 90b0999a5e..2fb0c8136f 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index eff853f379..e1b73bc25e 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import spark.{Logging, RDD, SparkContext} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index ea80bfcbfd..b864fd4634 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 4c18cbdc6b..7da96397a6 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.recommendation import scala.collection.mutable.{ArrayBuffer, BitSet} diff --git a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala index fb812a6dbe..38637b3dd1 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.recommendation import spark.RDD diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index e4db7bb9b7..bb294c2257 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import spark.{Logging, RDD, SparkContext} diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala index 6e7c023bac..8094d22405 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala index f79974c191..645204ddf3 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Regression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Regression.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import spark.RDD diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 5f813df402..7c7f912b43 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import spark.{Logging, RDD, SparkContext} diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala index b83f505d8e..c2260ae286 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala index 08a031dded..b5e564df6d 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.util import spark.{RDD, SparkContext} diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 390c92763c..a112e0b506 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file core/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the file core/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala index cb096f39a9..bebade9afb 100644 --- a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.clustering import scala.util.Random diff --git a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala index 2ada9ae76b..f98590b8d9 100644 --- a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.recommendation import scala.util.Random diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 04d3400cb4..bc9bfd054f 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala index df41dbbdff..3c588c6162 100644 --- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random diff --git a/pom.xml b/pom.xml index 6fa1f1d4f1..4b48072c6e 100644 --- a/pom.xml +++ b/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 org.spark-project diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 178a056de8..7a4d4c4575 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1,3 +1,19 @@ +/* + * 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. + */ import sbt._ import sbt.Classpaths.publishTask diff --git a/project/build.properties b/project/build.properties index 9b860e23c5..08e17131f6 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1,18 @@ +# +# 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. +# + sbt.version=0.12.3 diff --git a/pyspark b/pyspark index d662e90287..37a355462e 100755 --- a/pyspark +++ b/pyspark @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/python/epydoc.conf b/python/epydoc.conf index 45102cd9fe..d5d5aa5454 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -1,5 +1,22 @@ [epydoc] # Epydoc section marker (required by ConfigParser) +# +# 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. +# + # Information about the project. name: PySpark url: http://spark-project.org diff --git a/python/examples/als.py b/python/examples/als.py index 010f80097f..f2b2eee64c 100755 --- a/python/examples/als.py +++ b/python/examples/als.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ This example requires numpy (http://www.numpy.org/) """ diff --git a/python/examples/kmeans.py b/python/examples/kmeans.py index 72cf9f88c6..c670556f2b 100644 --- a/python/examples/kmeans.py +++ b/python/examples/kmeans.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ This example requires numpy (http://www.numpy.org/) """ diff --git a/python/examples/logistic_regression.py b/python/examples/logistic_regression.py index f13698a86f..54d227d0d3 100755 --- a/python/examples/logistic_regression.py +++ b/python/examples/logistic_regression.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ This example requires numpy (http://www.numpy.org/) """ diff --git a/python/examples/pi.py b/python/examples/pi.py index 127cba029b..33c026e824 100644 --- a/python/examples/pi.py +++ b/python/examples/pi.py @@ -1,3 +1,20 @@ +# +# 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. +# + import sys from random import random from operator import add diff --git a/python/examples/transitive_closure.py b/python/examples/transitive_closure.py index 7f85a1008e..40be3b5000 100644 --- a/python/examples/transitive_closure.py +++ b/python/examples/transitive_closure.py @@ -1,3 +1,20 @@ +# +# 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. +# + import sys from random import Random diff --git a/python/examples/wordcount.py b/python/examples/wordcount.py index 857160624b..41c846ba79 100644 --- a/python/examples/wordcount.py +++ b/python/examples/wordcount.py @@ -1,3 +1,20 @@ +# +# 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. +# + import sys from operator import add diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 3e9d7d36da..d367f91967 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ >>> from pyspark.context import SparkContext >>> sc = SparkContext('local', 'test') diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index def810dd46..dfdaba274f 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ >>> from pyspark.context import SparkContext >>> sc = SparkContext('local', 'test') diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 657fe6f989..2f741cb345 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -1,3 +1,20 @@ +# +# 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. +# + import os import shutil import sys diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 78c9457b84..eb18ec08c9 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -1,3 +1,20 @@ +# +# 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. +# + import os import signal import socket diff --git a/python/pyspark/files.py b/python/pyspark/files.py index 001b7a28b6..89bcbcfe06 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -1,3 +1,20 @@ +# +# 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. +# + import os diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 2329e536cc..e503fb7621 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -1,3 +1,20 @@ +# +# 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. +# + import os import sys from subprocess import Popen, PIPE diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index a9fec17a9d..c6a6b24c5a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1,3 +1,20 @@ +# +# 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. +# + from base64 import standard_b64encode as b64enc import copy from collections import defaultdict diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 5a95144983..fecacd1241 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -1,3 +1,20 @@ +# +# 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. +# + import struct import cPickle diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 54ff1bf8e7..cc8cd9e3c4 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ An interactive shell. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 1e34d47365..dfd841b10a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 379bbfd4c2..75d692beeb 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ Worker that receives input from Piped RDD. """ diff --git a/python/run-tests b/python/run-tests index a3a9ff5dcb..1ee947d414 100755 --- a/python/run-tests +++ b/python/run-tests @@ -1,6 +1,24 @@ #!/usr/bin/env bash -# Figure out where the Scala framework is installed +# +# 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. +# + + +# Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; cd ../; pwd)" FAILED=0 diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 7a7280313e..7c4e722cc1 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run index c54c9e97a0..0e5100e00b 100755 --- a/repl-bin/src/deb/bin/run +++ b/repl-bin/src/deb/bin/run @@ -1,6 +1,23 @@ #!/bin/bash -SCALA_VERSION=2.9.2 +# +# 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. +# + +SCALA_VERSION=2.9.3 # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor index 47b9cccdfe..d3c1c23413 100755 --- a/repl-bin/src/deb/bin/spark-executor +++ b/repl-bin/src/deb/bin/spark-executor @@ -1,5 +1,22 @@ #!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + FWDIR="$(cd `dirname $0`; pwd)" echo "Running spark-executor with framework dir = $FWDIR" exec $FWDIR/run spark.executor.MesosExecutorBackend diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell index 219c66eb0b..8b258543db 100755 --- a/repl-bin/src/deb/bin/spark-shell +++ b/repl-bin/src/deb/bin/spark-shell @@ -1,4 +1,21 @@ #!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + FWDIR="$(cd `dirname $0`; pwd)" exec $FWDIR/run spark.repl.Main "$@" diff --git a/repl/pom.xml b/repl/pom.xml index 92a2020b48..7d8da03254 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala index 0e9aa863b5..274bc585db 100644 --- a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.repl import java.io.{ByteArrayOutputStream, InputStream} diff --git a/repl/src/main/scala/spark/repl/Main.scala b/repl/src/main/scala/spark/repl/Main.scala index 58809ab646..d824d62fd1 100644 --- a/repl/src/main/scala/spark/repl/Main.scala +++ b/repl/src/main/scala/spark/repl/Main.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.repl import scala.collection.mutable.Set diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index cfb1a390e6..a6d33e69d2 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the repl/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the repl/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala index f46e6d8be4..80ae605558 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/spark/repl/ReplSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.repl import java.io._ diff --git a/run b/run index 6b5bc01a51..72ee76d722 100755 --- a/run +++ b/run @@ -1,5 +1,22 @@ #!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/run.cmd b/run.cmd index cc5605f8a9..c91764e617 100644 --- a/run.cmd +++ b/run.cmd @@ -1,2 +1,20 @@ @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 + cmd /V /E /C %~dp0run2.cmd %* diff --git a/run2.cmd b/run2.cmd index a9c4df180f..dc5e50931e 100644 --- a/run2.cmd +++ b/run2.cmd @@ -1,5 +1,22 @@ @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 + set SCALA_VERSION=2.9.3 rem Figure out where the Spark framework is installed diff --git a/sbt/sbt b/sbt/sbt index 523fbb346b..8c9a42824f 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,8 +1,28 @@ #!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + EXTRA_ARGS="" if [ "$MESOS_HOME" != "" ]; then EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java" fi + export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) export SPARK_TESTING=1 # To put test classes on classpath + java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd index ce3ae70174..56a16d77f2 100644 --- a/sbt/sbt.cmd +++ b/sbt/sbt.cmd @@ -1,5 +1,25 @@ @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 + set EXTRA_ARGS= if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\java + set SPARK_HOME=%~dp0.. + java -Xmx1200M -XX:MaxPermSize=200m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*" diff --git a/spark-executor b/spark-executor index b66c374ca8..feccbf5cc2 100755 --- a/spark-executor +++ b/spark-executor @@ -1,4 +1,22 @@ #!/bin/sh + +# +# 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. +# + FWDIR="`dirname $0`" echo "Running spark-executor with framework dir = $FWDIR" exec $FWDIR/run spark.executor.MesosExecutorBackend diff --git a/spark-shell b/spark-shell index a8e72143fb..5371fc540a 100755 --- a/spark-shell +++ b/spark-shell @@ -1,4 +1,22 @@ #!/bin/bash --posix + +# +# 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. +# + # # Shell script for starting the Spark Shell REPL # Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} @@ -6,6 +24,7 @@ # Options: # -c Set the number of cores for REPL to use # + FWDIR="`dirname $0`" for o in "$@"; do diff --git a/spark-shell.cmd b/spark-shell.cmd index 34697d52d7..ec65eabb74 100644 --- a/spark-shell.cmd +++ b/spark-shell.cmd @@ -1,4 +1,22 @@ @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 + set FWDIR=%~dp0 set SPARK_LAUNCH_WITH_SCALA=1 cmd /V /E /C %FWDIR%run2.cmd spark.repl.Main %* diff --git a/streaming/pom.xml b/streaming/pom.xml index 2fb5bbdeb5..7e6b06d772 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 450e48d66e..1e4c1e3742 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.{Logging, Utils} diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index 9be7926a4a..684d3abb56 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.dstream._ diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala index 6b0fade7c6..399ca1c63d 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import org.apache.hadoop.fs.Path diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala index 3b331956f5..c09a332d44 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import dstream.InputDStream diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala index c2135195d8..12a14e233d 100644 --- a/streaming/src/main/scala/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/spark/streaming/Duration.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.Utils diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/spark/streaming/Interval.scala index 6a8b81760e..b30cd969e9 100644 --- a/streaming/src/main/scala/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/spark/streaming/Interval.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming private[streaming] diff --git a/streaming/src/main/scala/spark/streaming/Job.scala b/streaming/src/main/scala/spark/streaming/Job.scala index 67bd8388bc..ceb3f92b65 100644 --- a/streaming/src/main/scala/spark/streaming/Job.scala +++ b/streaming/src/main/scala/spark/streaming/Job.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import java.util.concurrent.atomic.AtomicLong diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala index 7696c4a592..a31230689f 100644 --- a/streaming/src/main/scala/spark/streaming/JobManager.scala +++ b/streaming/src/main/scala/spark/streaming/JobManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index b159d26c02..d4cf2e568c 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 8d0a83d439..47bf07bee1 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.StreamingContext._ diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala index 1c4b22a898..252cc2a303 100644 --- a/streaming/src/main/scala/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import util.{ManualClock, RecurringTimer, Clock} diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 36b841af8f..ffd656227d 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import akka.actor.Props diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/spark/streaming/Time.scala index f14decf08b..ad5eab9dd2 100644 --- a/streaming/src/main/scala/spark/streaming/Time.scala +++ b/streaming/src/main/scala/spark/streaming/Time.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming /** @@ -52,4 +69,4 @@ case class Time(private val millis: Long) { object Time { val ordering = Ordering.by((time: Time) => time.millis) -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala index 4d93f0a5f7..7dcb1d713d 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.api.java import spark.streaming.{Duration, Time, DStream} @@ -82,4 +99,4 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM object JavaDStream { implicit def fromDStream[T: ClassManifest](dstream: DStream[T]): JavaDStream[T] = new JavaDStream[T](dstream) -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index 548809a359..3ab5c1fdde 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.api.java import java.util.{List => JList} @@ -296,4 +313,4 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def checkpoint(interval: Duration) = { dstream.checkpoint(interval) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index 30240cad98..ccd15563b0 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.api.java import java.util.{List => JList} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index ed7b789d98..b7720ad0ea 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.api.java import spark.streaming._ diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala index 4ef4bb7de1..99553d295d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.{RDD, Partitioner} diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala index 41c3af4694..095137092a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD @@ -16,4 +33,4 @@ class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T override def compute(validTime: Time): Option[RDD[T]] = { Some(rdd) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala index 41b9bd9461..de0536125d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala index e993164f99..9d8c5c3175 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala index cabd34f5f2..78d7117f0f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala index a69af60589..d13bebb10f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala index c9644b3a83..4906f503c2 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.StreamingContext diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala index ee69ea5177..7df537eb56 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala index b589cbd4d5..06fda6fe8e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala index 3c5d43a609..4dbdec459d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Time, Duration, StreamingContext, DStream} diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 55d2957be4..6ee588af15 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala index 848afecfad..af41a1b9ac 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala index 6055aa6a05..8d8a6161c6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala index 20818a0cab..3fda84a38a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 122a529bb7..344b41c4d0 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver} diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala index 3c2a81947b..33f7cd063f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.StreamingContext diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala index da224ad6f7..b269061b73 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala index 1b2fa56779..236f74f575 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala index 343b6915e7..96260501ab 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.StreamingContext._ diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala index 1f9548bfb8..83b57b27f7 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.{RDD, Partitioner} diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala index 1408af0afa..5877b10e0e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.StreamingContext diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala index db62955036..4b46613d5e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala index 99660d9dee..e7fbc5bbcf 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index ff7a58be45..f09a8b9f90 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark._ diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala index 00bad5da34..3eaa9a7e7f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala index cbf0c88108..fd24d61730 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index 036c95a860..2d9937eab8 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.receivers import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala index 5533c3cf1e..22d554e7e4 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.receivers import akka.actor.Actor diff --git a/streaming/src/main/scala/spark/streaming/util/Clock.scala b/streaming/src/main/scala/spark/streaming/util/Clock.scala index 974651f9f6..d9ac722df5 100644 --- a/streaming/src/main/scala/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/spark/streaming/util/Clock.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util private[streaming] diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index 426a9b6f71..8ce5d8daf5 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util import spark.{Logging, RDD} diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala index 03749d4a94..bf04120293 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util import spark.SparkContext diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala index bd0b0e74c1..5cc6ad9dee 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util import java.nio.ByteBuffer diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala index 8e10276deb..7ecc44236d 100644 --- a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util private[streaming] diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 4cf10582a9..3b93790baa 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming; import com.google.common.base.Optional; diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala index 64a7e7cbf9..f9d25db8da 100644 --- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import collection.mutable.{SynchronizedBuffer, ArrayBuffer} @@ -64,4 +81,4 @@ object JavaTestUtils extends JavaTestBase { object JavaCheckpointTestUtils extends JavaTestBase { override def actuallyWait = true -} \ No newline at end of file +} diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 59c445e63f..063529a9cb 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file streaming/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index 565089a853..67e3e0cd30 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.StreamingContext._ diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index 607dea77ec..8c639648f0 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import dstream.FileInputDStream diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala index a5fa7ab92d..7fc649fe27 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.Logging diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index b024fc9dcc..1c5419b16d 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import akka.actor.Actor diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala index ad6aa79d10..cb34b5a7cc 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.dstream.{InputDStream, ForEachDStream} diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala index 80d827706f..894b765fc6 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.StreamingContext._ From 4ff494de20c36151dc29a60825d67e094d14acd4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 17:26:48 -0700 Subject: [PATCH 083/221] Some missing license headers --- bin/compute-classpath.cmd | 17 +++++++++++++++++ bin/compute-classpath.sh | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 44826f339c..835d1af794 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -1,5 +1,22 @@ @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. diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index bd48b43da3..4c539649a2 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -1,5 +1,22 @@ #!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # This script computes Spark's classpath and prints it to stdout; it's used by both the "run" # script and the ExecutorRunner in standalone cluster mode. From a1d2c3434657f0d901a2f8cfddf62065b5ad4385 Mon Sep 17 00:00:00 2001 From: ctn Date: Wed, 17 Jul 2013 11:27:11 -0700 Subject: [PATCH 084/221] [BUGFIX] Fix for sbt/sbt script SPARK_HOME setting In some environments, this command export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) echoes two paths, one by the "cd ..", and one by the "pwd". Note the resulting erroneous -jar paths below: ctn@ubuntu:~/src/spark$ sbt/sbt + EXTRA_ARGS= + '[' '' '!=' '' ']' +++ dirname sbt/sbt ++ cd sbt/.. ++ pwd + export 'SPARK_HOME=/home/ctn/src/spark /home/ctn/src/spark' + SPARK_HOME='/home/ctn/src/spark /home/ctn/src/spark' + export SPARK_TESTING=1 + SPARK_TESTING=1 + java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m -jar /home/ctn/src/spark /home/ctn/src/spark/sbt/sbt-launch-0.11.3-2.jar Error: Invalid or corrupt jarfile /home/ctn/src/spark Committer: ctn On branch master Changes to be committed: - Send output of the "cd .." part to /dev/null modified: sbt/sbt --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 8c9a42824f..d64645e32d 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -22,7 +22,7 @@ if [ "$MESOS_HOME" != "" ]; then EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java" fi -export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) +export SPARK_HOME=$(cd "$(dirname $0)/.." 2>&1 >/dev/null ; pwd) export SPARK_TESTING=1 # To put test classes on classpath java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" From 84fa20c2a135f54745ddde9abb4f5e60af8856d1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:04:05 -0700 Subject: [PATCH 085/221] Allow initial weight vectors in LogisticRegression. Also move LogisticGradient to the LogisticRegression file and fix the unit tests log path. --- .../spark/mllib/optimization/Gradient.scala | 18 ------ .../mllib/optimization/GradientDescent.scala | 5 +- .../mllib/regression/LogisticRegression.scala | 61 ++++++++++++++++++- mllib/src/test/resources/log4j.properties | 2 +- .../regression/LogisticRegressionSuite.scala | 52 +++++++++++++--- 5 files changed, 106 insertions(+), 32 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 2fb0c8136f..d5338360c8 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -30,21 +30,3 @@ abstract class Gradient extends Serializable { def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): (DoubleMatrix, Double) } - -class LogisticGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { - val margin: Double = -1.0 * data.dot(weights) - val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - - val gradient = data.mul(gradientMultiplier) - val loss = - if (margin > 0) { - math.log(1 + math.exp(0 - margin)) - } else { - math.log(1 + math.exp(margin)) - margin - } - - (gradient, loss) - } -} diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index e1b73bc25e..2ac0808357 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -48,16 +48,17 @@ object GradientDescent { updater: Updater, stepSize: Double, numIters: Int, + initialWeights: Array[Double], miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { val lossHistory = new ArrayBuffer[Double](numIters) - val nfeatures: Int = data.take(1)(0)._2.length val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction // Initialize weights as a column matrix - var weights = DoubleMatrix.ones(nfeatures) + var weights = new DoubleMatrix(1, initialWeights.length, + initialWeights:_*) var reg_val = 0.0 for (i <- 1 to numIters) { diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index bb294c2257..dab15aa386 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -46,6 +46,24 @@ class LogisticRegressionModel( } } +class LogisticGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + val margin: Double = -1.0 * data.dot(weights) + val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label + + val gradient = data.mul(gradientMultiplier) + val loss = + if (margin > 0) { + math.log(1 + math.exp(0 - margin)) + } else { + math.log(1 + math.exp(margin)) - margin + } + + (gradient, loss) + } +} + class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -80,13 +98,30 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val initialWeights = Array.fill(nfeatures)(1.0) + train(input, initialWeights) + } + + def train( + input: RDD[(Double, Array[Double])], + initialWeights: Array[Double]): LogisticRegressionModel = { + // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => (y, Array(1.0, features:_*)) } + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) + val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + data, + new LogisticGradient(), + new SimpleUpdater(), + stepSize, + numIters, + initalWeightsWithIntercept, + miniBatchFraction) val weightsScaled = weights.getRange(1, weights.length) val intercept = weights.get(0) @@ -105,6 +140,30 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D */ object LogisticRegression { + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LogisticRegressionModel = + { + new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index a112e0b506..4265ba6e5d 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=ml/target/unit-tests.log +log4j.appender.file.file=mllib/target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index bc9bfd054f..2ff248d256 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -34,16 +34,14 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { - val nPoints = 10000 + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int) : Seq[(Double, Array[Double])] = { val rnd = new Random(42) - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val A = 2.0 - val B = -1.5 - // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) val unifRand = new scala.util.Random(45) val rLogis = (0 until nPoints).map { i => @@ -51,14 +49,24 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { math.log(u) - math.log(1.0-u) } - // y <- A + B*x + rlogis(100) + // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) val y = (0 until nPoints).map { i => - val yVal = A + B * x1(i) + rLogis(i) + val yVal = offset + scale * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))) + testData + } + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints) val testRDD = sc.parallelize(testData, 2) testRDD.cache() @@ -71,4 +79,28 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") } + + test("logistic regression with initial weights") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints) + + val initialB = -1.0 + val initialWeights = Array(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new LogisticRegression().setStepSize(10.0) + .setNumIterations(10) + + val model = lr.train(testRDD, initialWeights) + + val weight0 = model.weights.get(0) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + } } From 64b88e039a66fffdc1d0f0eb1caa9328d76bbb06 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:11:44 -0700 Subject: [PATCH 086/221] Move ML lib data generator files to util/ --- .../{regression => util}/LogisticRegressionGenerator.scala | 2 +- .../mllib/{regression => util}/RidgeRegressionGenerator.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename mllib/src/main/scala/spark/mllib/{regression => util}/LogisticRegressionGenerator.scala (98%) rename mllib/src/main/scala/spark/mllib/{regression => util}/RidgeRegressionGenerator.scala (98%) diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala index 8094d22405..4c580b44da 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression +package spark.mllib.util import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala index c2260ae286..a5bb2e762f 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression +package spark.mllib.util import scala.util.Random From 3bf989713654129ad35a80309d1b354ca5ddd06c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:20:24 -0700 Subject: [PATCH 087/221] Rename loss -> stochasticLoss and add a note to explain why we have multiple train methods. --- .../spark/mllib/optimization/GradientDescent.scala | 9 +++++---- .../spark/mllib/regression/LogisticRegression.scala | 10 ++++++---- .../scala/spark/mllib/regression/RidgeRegression.scala | 2 ++ 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 2ac0808357..77f5a7ae24 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -40,7 +40,8 @@ object GradientDescent { * one iteration of SGD. Default value 1.0. * * @return weights - Column matrix containing weights for every feature. - * @return lossHistory - Array containing the loss computed for every iteration. + * @return stochasticLossHistory - Array containing the stochastic loss computed for + * every iteration. */ def runMiniBatchSGD( data: RDD[(Double, Array[Double])], @@ -51,7 +52,7 @@ object GradientDescent { initialWeights: Array[Double], miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { - val lossHistory = new ArrayBuffer[Double](numIters) + val stochasticLossHistory = new ArrayBuffer[Double](numIters) val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction @@ -69,12 +70,12 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - lossHistory.append(lossSum / miniBatchSize + reg_val) + stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) weights = update._1 reg_val = update._2 } - (weights, lossHistory.toArray) + (weights, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index dab15aa386..664baf33a3 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -30,7 +30,7 @@ import org.jblas.DoubleMatrix class LogisticRegressionModel( val weights: DoubleMatrix, val intercept: Double, - val losses: Array[Double]) extends RegressionModel { + val stochasticLosses: Array[Double]) extends RegressionModel { override def predict(testData: spark.RDD[Array[Double]]) = { testData.map { x => @@ -114,7 +114,7 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val (weights, losses) = GradientDescent.runMiniBatchSGD( + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( data, new LogisticGradient(), new SimpleUpdater(), @@ -126,17 +126,19 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D val weightsScaled = weights.getRange(1, weights.length) val intercept = weights.get(0) - val model = new LogisticRegressionModel(weightsScaled, intercept, losses) + val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) logInfo("Final model weights " + model.weights) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochastic losses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling Logistic Regression. + * NOTE(shivaram): We use multiple train methods instead of default arguments to support + * Java programs. */ object LogisticRegression { diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 7c7f912b43..f724edd732 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -164,6 +164,8 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) /** * Top-level methods for calling Ridge Regression. + * NOTE(shivaram): We use multiple train methods instead of default arguments to support + * Java programs. */ object RidgeRegression { From 45f3c855181539306d5610c5aa265f24b431c142 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 16:03:29 -0700 Subject: [PATCH 088/221] Change weights to be Array[Double] in LR model. Also ensure weights are initialized to a column vector. --- .../mllib/optimization/GradientDescent.scala | 5 ++--- .../mllib/regression/LogisticRegression.scala | 17 +++++++++++------ .../regression/LogisticRegressionSuite.scala | 4 ++-- 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 77f5a7ae24..2c5038757b 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -57,9 +57,8 @@ object GradientDescent { val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction - // Initialize weights as a column matrix - var weights = new DoubleMatrix(1, initialWeights.length, - initialWeights:_*) + // Initialize weights as a column vector + var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) var reg_val = 0.0 for (i <- 1 to numIters) { diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index 664baf33a3..ab865af0c6 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -28,20 +28,23 @@ import org.jblas.DoubleMatrix * Based on Matlab code written by John Duchi. */ class LogisticRegressionModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, val stochasticLosses: Array[Double]) extends RegressionModel { + // Create a column vector that can be used for predictions + private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) + override def predict(testData: spark.RDD[Array[Double]]) = { testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(weightsMatrix).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - val margin = dataMat.mmul(this.weights).get(0) + this.intercept + val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } @@ -123,12 +126,14 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D initalWeightsWithIntercept, miniBatchFraction) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val weightsArray = weights.toArray() + + val intercept = weightsArray(0) + val weightsScaled = weightsArray.tail val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) - logInfo("Final model weights " + model.weights) + logInfo("Final model weights " + model.weights.mkString(",")) logInfo("Final model intercept " + model.intercept) logInfo("Last 10 stochastic losses " + model.stochasticLosses.takeRight(10).mkString(", ")) model diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 2ff248d256..47191d9a5a 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -75,7 +75,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val model = lr.train(testRDD) - val weight0 = model.weights.get(0) + val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") } @@ -99,7 +99,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val model = lr.train(testRDD, initialWeights) - val weight0 = model.weights.get(0) + val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") } From 217667174e267adba5469cf26b3e4418e3d1cc90 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 16:08:34 -0700 Subject: [PATCH 089/221] Return Array[Double] from SGD instead of DoubleMatrix --- .../scala/spark/mllib/optimization/GradientDescent.scala | 4 ++-- .../scala/spark/mllib/regression/LogisticRegression.scala | 6 ++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 2c5038757b..4c996c0903 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -50,7 +50,7 @@ object GradientDescent { stepSize: Double, numIters: Int, initialWeights: Array[Double], - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + miniBatchFraction: Double=1.0) : (Array[Double], Array[Double]) = { val stochasticLossHistory = new ArrayBuffer[Double](numIters) @@ -75,6 +75,6 @@ object GradientDescent { reg_val = update._2 } - (weights, stochasticLossHistory.toArray) + (weights.toArray, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index ab865af0c6..711e205c39 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -126,10 +126,8 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D initalWeightsWithIntercept, miniBatchFraction) - val weightsArray = weights.toArray() - - val intercept = weightsArray(0) - val weightsScaled = weightsArray.tail + val intercept = weights(0) + val weightsScaled = weights.tail val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) From 88a0823c58304fa1b290667d95a482105a8a5f52 Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Thu, 18 Jul 2013 00:51:18 +0000 Subject: [PATCH 090/221] Consistently invoke bash with /usr/bin/env bash in scripts to make code more portable (JIRA Ticket SPARK-817) --- bin/compute-classpath.sh | 2 +- docs/spark-simple-tutorial.md | 2 +- ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh | 2 +- ec2/deploy.generic/root/spark-ec2/ec2-variables.sh | 2 +- repl-bin/src/deb/bin/run | 2 +- repl-bin/src/deb/bin/spark-executor | 2 +- repl-bin/src/deb/bin/spark-shell | 2 +- run | 2 +- sbt/sbt | 2 +- spark-shell | 5 ++++- 10 files changed, 13 insertions(+), 10 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 4c539649a2..e6fdc5306e 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/docs/spark-simple-tutorial.md b/docs/spark-simple-tutorial.md index 9875de62bd..fbdbc7d19d 100644 --- a/docs/spark-simple-tutorial.md +++ b/docs/spark-simple-tutorial.md @@ -13,7 +13,7 @@ title: Tutorial - Running a Simple Spark Application 3. Edit the ~/SparkTest/sbt/sbt file to look like this: - #!/bin/bash + #!/usr/bin/env bash java -Xmx800M -XX:MaxPermSize=150m -jar $(dirname $0)/sbt-launch-*.jar "$@" 4. To build a Spark application, you need Spark and its dependencies in a single Java archive (JAR) file. Create this JAR in Spark's main directory with sbt as: diff --git a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh b/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh index 50ecf83404..ede6c78428 100644 --- a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # These variables are automatically filled in by the mesos-ec2 script. export MESOS_MASTERS="{{master_list}}" diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 166a884c88..685ed8be8c 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # These variables are automatically filled in by the mesos-ec2 script. export MESOS_MASTERS="{{master_list}}" diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run index 0e5100e00b..8b5d8300f2 100755 --- a/repl-bin/src/deb/bin/run +++ b/repl-bin/src/deb/bin/run @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor index d3c1c23413..bcfae22677 100755 --- a/repl-bin/src/deb/bin/spark-executor +++ b/repl-bin/src/deb/bin/spark-executor @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell index 8b258543db..ec7e33e1e3 100755 --- a/repl-bin/src/deb/bin/spark-shell +++ b/repl-bin/src/deb/bin/spark-shell @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/run b/run index 72ee76d722..f49aa92796 100755 --- a/run +++ b/run @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/sbt/sbt b/sbt/sbt index 8c9a42824f..b6f91f5085 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/spark-shell b/spark-shell index 5371fc540a..31a4138124 100755 --- a/spark-shell +++ b/spark-shell @@ -1,4 +1,4 @@ -#!/bin/bash --posix +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -25,6 +25,9 @@ # -c Set the number of cores for REPL to use # +# Enter posix mode for bash +set -o posix + FWDIR="`dirname $0`" for o in "$@"; do From 3aad45265324537e51cab79db3f09d9310339f77 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 02:29:46 +0800 Subject: [PATCH 091/221] fix a bug in build process that pulls in two versionf of ASM. --- project/SparkBuild.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e9819b877d..59edda5097 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -183,10 +183,10 @@ object SparkBuild extends Build { if (HADOOP_YARN) { Seq( // Exclude rule required for all ? - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm) ) } else { Seq( From 7ab1170503ab46ffd838fb98590dd3d66748dd38 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 18 Jul 2013 11:55:19 -0700 Subject: [PATCH 092/221] Refactor data generators to have a function that can be used in tests. --- .../util/LogisticRegressionGenerator.scala | 48 ++++++++----- .../mllib/util/RidgeRegressionGenerator.scala | 67 ++++++++++++------- 2 files changed, 76 insertions(+), 39 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala index 4c580b44da..8d659cd97c 100644 --- a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala @@ -19,12 +19,39 @@ package spark.mllib.util import scala.util.Random -import org.jblas.DoubleMatrix - import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils -object LogisticRegressionGenerator { +object LogisticRegressionDataGenerator { + + /** + * Generate an RDD containing test data for LogisticRegression. This function chooses + * positive labels with probability `probOne` and scales positive examples by `eps`. + * + * @param sc SparkContext to use for creating the RDD. + * @param nexamples Number of examples that will be contained in the RDD. + * @param nfeatures Number of features to generate for each example. + * @param eps Epsilon factor by which positive examples are scaled. + * @param nparts Number of partitions of the generated RDD. Default value is 2. + * @param probOne Probability that a label is 1 (and not 0). Default value is 0.5. + */ + def generateLogisticRDD( + sc: SparkContext, + nexamples: Int, + nfeatures: Int, + eps: Double, + nparts: Int = 2, + probOne: Double = 0.5): RDD[(Double, Array[Double])] = { + val data = sc.parallelize(0 until nexamples, nparts).map { idx => + val rnd = new Random(42 + idx) + + val y = if (idx % 2 == 0) 0.0 else 1.0 + val x = Array.fill[Double](nfeatures) { + rnd.nextGaussian() + (y * eps) + } + (y, x) + } + data + } def main(args: Array[String]) { if (args.length != 5) { @@ -40,17 +67,8 @@ object LogisticRegressionGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 3 - val sc = new SparkContext(sparkMaster, "LogisticRegressionGenerator") - - val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => - val rnd = new Random(42 + idx) - - val y = if (idx % 2 == 0) 0 else 1 - val x = Array.fill[Double](nfeatures) { - rnd.nextGaussian() + (y * eps) - } - (y, x) - } + val sc = new SparkContext(sparkMaster, "LogisticRegressionDataGenerator") + val data = generateLogisticRDD(sc, nexamples, nfeatures, eps, parts) MLUtils.saveLabeledData(data, outputPath) sc.stop() diff --git a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala index a5bb2e762f..6861913dc7 100644 --- a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala @@ -22,11 +22,52 @@ import scala.util.Random import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils object RidgeRegressionGenerator { + /** + * Generate an RDD containing test data used for RidgeRegression. This function generates + * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the + * response variable `Y`. + * + * @param sc SparkContext to be used for generating the RDD. + * @param nexamples Number of examples that will be contained in the RDD. + * @param nfeatures Number of features to generate for each example. + * @param eps Epsilon factor by which examples are scaled. + * @param nparts Number of partitions in the RDD. Default value is 2. + */ + def generateRidgeRDD( + sc: SparkContext, + nexamples: Int, + nfeatures: Int, + eps: Double, + nparts: Int = 2) : RDD[(Double, Array[Double])] = { + org.jblas.util.Random.seed(42) + // Random values distributed uniformly in [-0.5, 0.5] + val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) + w.put(0, 0, 10) + w.put(1, 0, 10) + + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nparts, nparts).flatMap { p => + org.jblas.util.Random.seed(42 + p) + val examplesInPartition = nexamples / nparts + + val X = DoubleMatrix.rand(examplesInPartition, nfeatures) + val y = X.mmul(w) + + val rnd = new Random(42 + p) + + val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) + val yObs = new DoubleMatrix(normalValues).addi(y) + + Iterator.tabulate(examplesInPartition) { i => + (yObs.get(i, 0), X.getRow(i).toArray) + } + } + data + } + def main(args: Array[String]) { if (args.length != 5) { println("Usage: RidgeRegressionGenerator " + @@ -41,30 +82,8 @@ object RidgeRegressionGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 10 - org.jblas.util.Random.seed(42) val sc = new SparkContext(sparkMaster, "RidgeRegressionGenerator") - - // Random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - w.put(0, 0, 10) - w.put(1, 0, 10) - - val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until parts, parts).flatMap { p => - org.jblas.util.Random.seed(42 + p) - val examplesInPartition = nexamples / parts - - val X = DoubleMatrix.rand(examplesInPartition, nfeatures) - val y = X.mmul(w) - - val rnd = new Random(42 + p) - - val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) - val yObs = new DoubleMatrix(normalValues).addi(y) - - Iterator.tabulate(examplesInPartition) { i => - (yObs.get(i, 0), X.getRow(i).toArray) - } - } + val data = generateRidgeRDD(sc, nexamples, nfeatures, eps, parts) MLUtils.saveLabeledData(data, outputPath) sc.stop() From 2c9ea56db4a1f4324f1dc8fadaf2491d83eda8ca Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 18 Jul 2013 11:57:14 -0700 Subject: [PATCH 093/221] Rename classes to be called DataGenerator --- ...Generator.scala => LogisticRegressionDataGenerator.scala} | 0 ...ionGenerator.scala => RidgeRegressionDataGenerator.scala} | 5 ++--- 2 files changed, 2 insertions(+), 3 deletions(-) rename mllib/src/main/scala/spark/mllib/util/{LogisticRegressionGenerator.scala => LogisticRegressionDataGenerator.scala} (100%) rename mllib/src/main/scala/spark/mllib/util/{RidgeRegressionGenerator.scala => RidgeRegressionDataGenerator.scala} (96%) diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala index 6861913dc7..c5b8a29942 100644 --- a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala @@ -23,8 +23,7 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} - -object RidgeRegressionGenerator { +object RidgeRegressionDataGenerator { /** * Generate an RDD containing test data used for RidgeRegression. This function generates @@ -82,7 +81,7 @@ object RidgeRegressionGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 10 - val sc = new SparkContext(sparkMaster, "RidgeRegressionGenerator") + val sc = new SparkContext(sparkMaster, "RidgeRegressionDataGenerator") val data = generateRidgeRDD(sc, nexamples, nfeatures, eps, parts) MLUtils.saveLabeledData(data, outputPath) From a613628c5078cf41feb973d0ee8a06eb69615bcf Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 16:59:12 +0800 Subject: [PATCH 094/221] Do not copy local jars given to SparkContext in yarn mode since the Context is not running on local. This bug causes failure when jars can not be found. Example codes (such as spark.examples.SparkPi) can not work without this fix under yarn mode. --- core/src/main/scala/spark/SparkContext.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 46b9935cb7..957c541ecf 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -102,6 +102,7 @@ class SparkContext( } private val isLocal = (master == "local" || master.startsWith("local[")) + private val isYarn = (master == "yarn-standalone") // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( @@ -577,11 +578,18 @@ class SparkContext( } else { val uri = new URI(path) val key = uri.getScheme match { - case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) + case null | "file" => + if (!isYarn) + env.httpFileServer.addJar(new File(uri.getPath)) + else + null case _ => path } - addedJars(key) = System.currentTimeMillis - logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) + + if (key != null) { + addedJars(key) = System.currentTimeMillis + logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) + } } } From aa6f83289b87f38481dbae60ad91d2ac78ccea46 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 22:25:28 +0800 Subject: [PATCH 095/221] A better fix for giving local jars unde Yarn mode. --- .../scala/spark/deploy/SparkHadoopUtil.scala | 3 +++ .../scala/spark/deploy/SparkHadoopUtil.scala | 3 +++ core/src/main/scala/spark/SparkContext.scala | 17 +++++++---------- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index df55be1254..9f040faac3 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -41,4 +41,7 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} + + def isYarnMode(): Boolean = { False } + } diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index df55be1254..9f040faac3 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -41,4 +41,7 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} + + def isYarnMode(): Boolean = { False } + } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 957c541ecf..c01e315e35 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -102,7 +102,6 @@ class SparkContext( } private val isLocal = (master == "local" || master.startsWith("local[")) - private val isYarn = (master == "yarn-standalone") // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( @@ -579,17 +578,15 @@ class SparkContext( val uri = new URI(path) val key = uri.getScheme match { case null | "file" => - if (!isYarn) - env.httpFileServer.addJar(new File(uri.getPath)) - else - null + if (SparkHadoopUtil.isYarnMode()) { + logWarning("local jar specified as parameter to addJar under Yarn mode") + return + } + env.httpFileServer.addJar(new File(uri.getPath)) case _ => path } - - if (key != null) { - addedJars(key) = System.currentTimeMillis - logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) - } + addedJars(key) = System.currentTimeMillis + logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } } From 4530e8a9bfe35b6d562876b1fb66e534ff5c286d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Jul 2013 00:04:25 +0800 Subject: [PATCH 096/221] fix typo. --- core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala | 2 +- core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index 9f040faac3..617954cb98 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -42,6 +42,6 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} - def isYarnMode(): Boolean = { False } + def isYarnMode(): Boolean = { false } } diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index 9f040faac3..617954cb98 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -42,6 +42,6 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} - def isYarnMode(): Boolean = { False } + def isYarnMode(): Boolean = { false } } From d1738d72ba2ff28c38e03beb3e17f03d3dd77e1a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Jul 2013 00:37:24 +0800 Subject: [PATCH 097/221] also exclude asm for hadoop2. hadoop1 looks like no need to do that too. --- project/SparkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 59edda5097..2f2cbf646a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -190,8 +190,8 @@ object SparkBuild extends Build { ) } else { Seq( - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm) ) } } else { From cfce9a6a365a8a4b156e99f412157704cab592b9 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Thu, 18 Jul 2013 17:43:27 -0700 Subject: [PATCH 098/221] Regression: default webui-port can't be set via command line "--webui-port" anymore --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- .../main/scala/spark/deploy/master/MasterArguments.scala | 5 ++++- .../src/main/scala/spark/deploy/master/ui/MasterWebUI.scala | 6 ++---- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index e5a7a87e2e..eddcafd84d 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -53,7 +53,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act var firstApp: Option[ApplicationInfo] = None - val webUi = new MasterWebUI(self) + val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala index d0ec3d5ea0..0ae0160767 100644 --- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala @@ -38,7 +38,10 @@ private[spark] class MasterArguments(args: Array[String]) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } - + if (System.getProperty("master.ui.port") != null) { + webUiPort = System.getProperty("master.ui.port").toInt + } + parse(args.toList) def parse(args: List[String]): Unit = args match { diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index 04b32c7968..dabc2d8dc7 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -32,12 +32,11 @@ import spark.ui.JettyUtils._ * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) extends Logging { +class MasterWebUI(val master: ActorRef, requestedPort: Int) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() - val port = requestedPort.getOrElse( - System.getProperty("master.ui.port", MasterWebUI.DEFAULT_PORT).toInt) + val port = requestedPort var server: Option[Server] = None var boundPort: Option[Int] = None @@ -72,5 +71,4 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten private[spark] object MasterWebUI { val STATIC_RESOURCE_DIR = "spark/ui/static" - val DEFAULT_PORT = "8080" } From 81bb5dc64007121df52cbafdf5f025c97f145953 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 19 Jul 2013 14:08:30 -0700 Subject: [PATCH 099/221] Creates Executors tab for application with RDD block and memory/disk used, solves SPARK-808 --- .../main/scala/spark/executor/Executor.scala | 1 + .../scala/spark/executor/TaskMetrics.scala | 5 ++ core/src/main/scala/spark/ui/Page.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 5 +- core/src/main/scala/spark/ui/UIUtils.scala | 5 ++ .../scala/spark/ui/exec/ExecutorsUI.scala | 79 +++++++++++++++++++ .../scala/spark/ui/storage/IndexPage.scala | 18 +---- 7 files changed, 96 insertions(+), 19 deletions(-) create mode 100644 core/src/main/scala/spark/ui/exec/ExecutorsUI.scala diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 2e81151882..82e5f5a015 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -126,6 +126,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => m.hostname = Utils.localHostName + m.executorId = executorId m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 3151627839..e14dc57501 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -23,6 +23,11 @@ class TaskMetrics extends Serializable { */ var hostname: String = _ + /** + * Executor's ID the task runs on + */ + var executorId: String = _ + /** * Time taken on the executor to deserialize this task */ diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala index a31e750d06..03034a4520 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/spark/ui/Page.scala @@ -17,4 +17,4 @@ package spark.ui -private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } +private[spark] object Page extends Enumeration { val Storage, Jobs, Environment, Executors = Value } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 9396f22063..1534705bc5 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -23,6 +23,7 @@ import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, SparkContext, Utils} import spark.ui.env.EnvironmentUI +import spark.ui.exec.ExecutorsUI import spark.ui.storage.BlockManagerUI import spark.ui.jobs.JobProgressUI import spark.ui.JettyUtils._ @@ -41,7 +42,9 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) val env = new EnvironmentUI(sc) - val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ handlers + val exec = new ExecutorsUI(sc) + val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ + exec.getHandlers ++ handlers /** Bind the HTTP server which backs this web interface */ def bind() { diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index b1d11954dd..e33c80282a 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -40,6 +40,10 @@ private[spark] object UIUtils { case Environment =>
      • Environment
      • case _ =>
      • Environment
      • } + val executors = page match { + case Executors =>
      • Executors
      • + case _ =>
      • Executors
      • + } @@ -66,6 +70,7 @@ private[spark] object UIUtils { {storage} {jobs} {environment} + {executors}
      • Application: {sc.appName}
      • diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala new file mode 100644 index 0000000000..de0e571553 --- /dev/null +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -0,0 +1,79 @@ +package spark.ui.exec + + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.util.Properties + +import spark.scheduler.cluster.TaskInfo +import spark.executor.TaskMetrics +import spark.ui.JettyUtils._ +import spark.ui.UIUtils.headerSparkPage +import spark.ui.Page.Executors +import spark.storage.{StorageStatus, StorageUtils} +import spark.SparkContext +import spark.ui.UIUtils +import spark.Utils + +import scala.xml.{Node, XML} + +private[spark] class ExecutorsUI(val sc: SparkContext) { + + def getHandlers = Seq[(String, Handler)]( + ("/executors", (request: HttpServletRequest) => render(request)) + ) + + def render(request: HttpServletRequest): Seq[Node] = { + val storageStatusList = sc.getExecutorStorageStatus + + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + + val execTables = + for (a <- 0 until storageStatusList.size) + yield getExecTable(a) + + val content = +
        +
        +
          +
        • Memory: + {Utils.memoryBytesToString(maxMem - remainingMem)} Used + ({Utils.memoryBytesToString(remainingMem)} Available)
        • +
        • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
        • +
        +
        +
        +
        +
        + {execTables} +
        +
        ; + + headerSparkPage(content, sc, "Executors", Executors) + } + + def getExecTable(a: Int): Seq[Node] = { + val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) + val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) + val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) + val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString + val execInfo = Seq( + ("RDD blocks", rddBlocks), + ("Memory used", "%s/%s".format(memUsed, maxMem)), + ("Disk used", diskUsed) + ) + def execRow(kv: (String, String)) = {kv._1}{kv._2} + val table = UIUtils.listingTable(Seq("Name", "Value"), execRow, execInfo) + val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId + val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort + val header = +

        Executor {execId}

        +

        {hostPort}

        ; + header ++ table + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 4e0360d19a..f76192eba8 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -33,10 +33,6 @@ private[spark] class IndexPage(parent: BlockManagerUI) { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) val rddHeaders = Seq( "RDD Name", @@ -46,19 +42,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) { "Size in Memory", "Size on Disk") val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val rddTable = listingTable(rddHeaders, rddRow, rdds) - - val content = -
        -
        -
          -
        • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
        • -
        • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
        • -
        -
        -
        ++ {rddTable}; + val content = listingTable(rddHeaders, rddRow, rdds) headerSparkPage(content, parent.sc, "Spark Storage ", Storage) } From 865dc63bac1cb72ecb84038c0a7cd708cc9e19d7 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 19 Jul 2013 15:57:01 -0700 Subject: [PATCH 100/221] Changed table format for executors --- .../scala/spark/ui/exec/ExecutorsUI.scala | 40 +++++++++++-------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index de0e571553..fa516a4a34 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -33,9 +33,19 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - val execTables = - for (a <- 0 until storageStatusList.size) - yield getExecTable(a) + val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used") + def execRow(kv: Seq[String]) = + + {kv(0)} + {kv(1)} + {kv(2)} + {kv(3)} + {kv(4)} + + val execInfo = + for (b <- 0 until storageStatusList.size) + yield getExecInfo(b) + val execTable = UIUtils.listingTable(execHead, execRow, execInfo) val content =
        @@ -50,30 +60,26 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
        - {execTables} + {execTable}
        ; headerSparkPage(content, sc, "Executors", Executors) } - def getExecTable(a: Int): Seq[Node] = { + def getExecInfo(a: Int): Seq[String] = { + val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId + val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString - val execInfo = Seq( - ("RDD blocks", rddBlocks), - ("Memory used", "%s/%s".format(memUsed, maxMem)), - ("Disk used", diskUsed) + Seq( + execId, + hostPort, + rddBlocks, + "%s/%s".format(memUsed, maxMem), + diskUsed ) - def execRow(kv: (String, String)) = {kv._1}{kv._2} - val table = UIUtils.listingTable(Seq("Name", "Value"), execRow, execInfo) - val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId - val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val header = -

        Executor {execId}

        -

        {hostPort}

        ; - header ++ table } } \ No newline at end of file From f4d514810e6fd9f42868ebb9a89390c62c3b42e1 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Tue, 2 Jul 2013 17:03:24 -0700 Subject: [PATCH 101/221] Building spark assembly for further consumption of the Spark project with a deployed cluster --- assembly/README | 13 +++ assembly/pom.xml | 104 ++++++++++++++++++++++++ assembly/src/main/assembly/assembly.xml | 68 ++++++++++++++++ pom.xml | 21 ++++- spark-shell | 1 + 5 files changed, 206 insertions(+), 1 deletion(-) create mode 100644 assembly/README create mode 100644 assembly/pom.xml create mode 100644 assembly/src/main/assembly/assembly.xml diff --git a/assembly/README b/assembly/README new file mode 100644 index 0000000000..6ee2a536d7 --- /dev/null +++ b/assembly/README @@ -0,0 +1,13 @@ +This is an assembly module for Spark project. + +It creates a single tar.gz file that includes all needed dependency of the project +except for org.apache.hadoop.* jars that are supposed to be available from the +deployed Hadoop cluster. + +This module is off by default to avoid spending extra time on top of repl-bin +module. To activate it specify the profile in the command line + -Passembly + +In case you want to avoid building time-expensive repl-bin module, that shaders +all the dependency into a big flat jar supplement maven command with + -DnoExpensive diff --git a/assembly/pom.xml b/assembly/pom.xml new file mode 100644 index 0000000000..1382539f24 --- /dev/null +++ b/assembly/pom.xml @@ -0,0 +1,104 @@ + + + 4.0.0 + + org.spark-project + spark-parent + 0.8.0-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-assembly + Spark Project Assembly + http://spark-project.org/ + + + + + org.apache.maven.plugins + maven-assembly-plugin + 2.4 + + + dist + package + + single + + + + src/main/assembly/assembly.xml + + + + + + + + + + + hadoop1 + + hadoop1 + + + + hadoop2 + + hadoop2 + + + + hadoop2-yarn + + hadoop2-yarn + + + + + + org.spark-project + spark-core + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-bagel + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-examples + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-examples + javadoc + 0.8.0-SNAPSHOT + + + org.spark-project + spark-examples + sources + 0.8.0-SNAPSHOT + + + org.spark-project + spark-repl + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-streaming + ${classifier.name} + 0.8.0-SNAPSHOT + + + \ No newline at end of file diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml new file mode 100644 index 0000000000..dd05f35f1f --- /dev/null +++ b/assembly/src/main/assembly/assembly.xml @@ -0,0 +1,68 @@ + + dist + + tar.gz + dir + + false + + + + + README + + + + + ${project.parent.basedir}/core/src/main/resources/spark/ui/static/ + + /ui-resources/spark/ui/static + + **/* + + + + + ${project.parent.basedir}/bin/ + + /bin + + **/* + + + + + ${project.parent.basedir} + + /bin + + run* + spark-shell* + spark-executor* + + + + + + + + org.spark-project:*:jar + + + org.spark-project:spark-dist:jar + + + + lib + true + false + runtime + false + + org.apache.hadoop:*:jar + org.spark-project:*:jar + + + + + diff --git a/pom.xml b/pom.xml index 4b48072c6e..6d9437b69f 100644 --- a/pom.xml +++ b/pom.xml @@ -60,7 +60,6 @@ examples streaming repl - repl-bin @@ -618,5 +617,25 @@ + + assembly + + false + + + assembly + + + + expensive-modules + + + !noExpensive + + + + repl-bin + + diff --git a/spark-shell b/spark-shell index 31a4138124..62fc18550d 100755 --- a/spark-shell +++ b/spark-shell @@ -79,6 +79,7 @@ if [[ ! $? ]]; then saved_stty="" fi +export SPARK_LAUNCH_WITH_SCALA=${SPARK_LAUNCH_WITH_SCALA:-1} $FWDIR/run $OPTIONS spark.repl.Main "$@" # record the exit status lest it be overwritten: From 0337d88321f3681009de548ce10ba7e0ca8f1a58 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Sun, 21 Jul 2013 18:07:19 -0700 Subject: [PATCH 102/221] Add a public method getCachedRdds to SparkContext --- core/src/main/scala/spark/SparkContext.scala | 8 +++++++- core/src/test/scala/spark/RDDSuite.scala | 6 ++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index c01e315e35..1b46665d2c 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -546,6 +546,12 @@ class SparkContext( StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) } + /** + * Returns an immutable map of RDDs that have marked themselves as cached via cache() call. + * Note that this does not necessarily mean the caching or computation was successful. + */ + def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.asInstanceOf[Map[Int, RDD[_]]] + def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos } @@ -580,7 +586,7 @@ class SparkContext( case null | "file" => if (SparkHadoopUtil.isYarnMode()) { logWarning("local jar specified as parameter to addJar under Yarn mode") - return + return } env.httpFileServer.addJar(new File(uri.getPath)) case _ => path diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index cbddf4e523..ff2dcd72d8 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -90,15 +90,19 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("basic caching") { + val origCachedRdds = sc.getCachedRDDs.size val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) + // Should only result in one cached RDD + assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("caching with failures") { val onlySplit = new Partition { override def index: Int = 0 } var shouldFail = true + val origCachedRdds = sc.getCachedRDDs.size val rdd = new RDD[Int](sc, Nil) { override def getPartitions: Array[Partition] = Array(onlySplit) override val getDependencies = List[Dependency[_]]() @@ -110,12 +114,14 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } }.cache() + assert(sc.getCachedRDDs.size === origCachedRdds + 1) val thrown = intercept[Exception]{ rdd.collect() } assert(thrown.getMessage.contains("injected failure")) shouldFail = false assert(rdd.collect().toList === List(1, 2, 3, 4)) + assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("empty RDD") { From 8901f379c98d19d3145f0cb2ae678d1b0060a311 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 09:58:03 -0700 Subject: [PATCH 103/221] Fixed memory used/remaining/total bug --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index fa516a4a34..5fb75c60fd 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -30,10 +30,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val memUsed = storageStatusList.map(_.memUsed()).reduce(_+_) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used") + val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used/Memory total", + "Disk used") def execRow(kv: Seq[String]) = {kv(0)} @@ -52,8 +54,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
        • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
        • + {Utils.memoryBytesToString(memUsed)} Used + ({Utils.memoryBytesToString(maxMem)} Total)
        • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
        @@ -70,8 +72,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { def getExecInfo(a: Int): Seq[String] = { val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) - val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) + val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) + val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString Seq( From f649dabb4a3a57cb25a852808297fb718cbfedd4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Jul 2013 13:14:33 -0700 Subject: [PATCH 104/221] Fix bug: DoubleRDDFunctions.sampleStdev() computed non-sample stdev(). Update JavaDoubleRDD to add new methods and docs. Fixes SPARK-825. --- .../main/scala/spark/DoubleRDDFunctions.scala | 8 ++++- .../scala/spark/api/java/JavaDoubleRDD.scala | 33 ++++++++++++++----- core/src/test/scala/spark/JavaAPISuite.java | 2 ++ 3 files changed, 33 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala index 93ef097702..104168e61c 100644 --- a/core/src/main/scala/spark/DoubleRDDFunctions.scala +++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala @@ -54,7 +54,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * Compute the sample standard deviation of this RDD's elements (which corrects for bias in * estimating the standard deviation by dividing by N-1 instead of N). */ - def sampleStdev(): Double = stats().stdev + def sampleStdev(): Double = stats().sampleStdev + + /** + * Compute the sample variance of this RDD's elements (which corrects for bias in + * estimating the variance by dividing by N-1 instead of N). + */ + def sampleVariance(): Double = stats().sampleVariance /** (Experimental) Approximate operation to return the mean within a timeout. */ def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 392556f261..8ce7df6213 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -115,33 +115,48 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav // Double RDD functions - /** Return the sum of the elements in this RDD. */ + /** Add up the elements in this RDD. */ def sum(): Double = srdd.sum() - /** Return a [[spark.StatCounter]] describing the elements in this RDD. */ + /** + * Return a [[spark.util.StatCounter]] object that captures the mean, variance and count + * of the RDD's elements in one operation. + */ def stats(): StatCounter = srdd.stats() - /** Return the mean of the elements in this RDD. */ + /** Compute the mean of this RDD's elements. */ def mean(): Double = srdd.mean() - /** Return the variance of the elements in this RDD. */ + /** Compute the variance of this RDD's elements. */ def variance(): Double = srdd.variance() - /** Return the standard deviation of the elements in this RDD. */ + /** Compute the standard deviation of this RDD's elements. */ def stdev(): Double = srdd.stdev() + /** + * Compute the sample standard deviation of this RDD's elements (which corrects for bias in + * estimating the standard deviation by dividing by N-1 instead of N). + */ + def sampleStdev(): Double = srdd.sampleStdev() + + /** + * Compute the sample variance of this RDD's elements (which corrects for bias in + * estimating the standard variance by dividing by N-1 instead of N). + */ + def sampleVariance(): Double = srdd.sampleVariance() + /** Return the approximate mean of the elements in this RDD. */ def meanApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = srdd.meanApprox(timeout, confidence) - /** Return the approximate mean of the elements in this RDD. */ + /** (Experimental) Approximate operation to return the mean within a timeout. */ def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout) - /** Return the approximate sum of the elements in this RDD. */ + /** (Experimental) Approximate operation to return the sum within a timeout. */ def sumApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - - /** Return the approximate sum of the elements in this RDD. */ + + /** (Experimental) Approximate operation to return the sum within a timeout. */ def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) } diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index aaf03e683b..5e2bf2d231 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -326,7 +326,9 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(20/6.0, rdd.mean(), 0.01); Assert.assertEquals(20/6.0, rdd.mean(), 0.01); Assert.assertEquals(6.22222, rdd.variance(), 0.01); + Assert.assertEquals(7.46667, rdd.sampleVariance(), 0.01); Assert.assertEquals(2.49444, rdd.stdev(), 0.01); + Assert.assertEquals(2.73252, rdd.sampleStdev(), 0.01); Double first = rdd.first(); List take = rdd.take(5); From 85c4d7bf3bf0969f58ebcda6ca68719972ff0c46 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 14:35:47 -0700 Subject: [PATCH 105/221] Shows number of complete/total/failed tasks (bug: failed tasks assigned to null executor) --- core/src/main/scala/spark/ui/SparkUI.scala | 1 + .../scala/spark/ui/exec/ExecutorsUI.scala | 65 ++++++++++++++++--- 2 files changed, 56 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 1534705bc5..7599f82a94 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -67,6 +67,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { // This server must register all handlers, including JobProgressUI, before binding // JobProgressUI registers a listener with SparkContext, which requires sc to initialize jobs.start() + exec.start() } def stop() { diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 5fb75c60fd..a981c680d2 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -5,15 +5,18 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.Properties -import spark.scheduler.cluster.TaskInfo +import spark.{ExceptionFailure, Logging, SparkContext, Success, Utils} import spark.executor.TaskMetrics -import spark.ui.JettyUtils._ -import spark.ui.UIUtils.headerSparkPage -import spark.ui.Page.Executors -import spark.storage.{StorageStatus, StorageUtils} +import spark.scheduler.cluster.TaskInfo +import spark.scheduler._ import spark.SparkContext +import spark.storage.{StorageStatus, StorageUtils} +import spark.ui.JettyUtils._ +import spark.ui.Page.Executors +import spark.ui.UIUtils.headerSparkPage import spark.ui.UIUtils import spark.Utils @@ -21,6 +24,14 @@ import scala.xml.{Node, XML} private[spark] class ExecutorsUI(val sc: SparkContext) { + private var _listener: Option[ExecutorsListener] = None + def listener = _listener.get + + def start() { + _listener = Some(new ExecutorsListener) + sc.addSparkListener(listener) + } + def getHandlers = Seq[(String, Handler)]( ("/executors", (request: HttpServletRequest) => render(request)) ) @@ -29,13 +40,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val storageStatusList = sc.getExecutorStorageStatus val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) val memUsed = storageStatusList.map(_.memUsed()).reduce(_+_) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used/Memory total", - "Disk used") + val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", + "Tasks: Complete/Total") def execRow(kv: Seq[String]) = {kv(0)} @@ -43,6 +53,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {kv(2)} {kv(3)} {kv(4)} + {kv(5)} val execInfo = for (b <- 0 until storageStatusList.size) @@ -76,12 +87,46 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString + val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0) + val totalTasks = listener.executorToTaskInfos(a.toString).size + val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0) match { + case f if f > 0 => " (%s failed)".format(f) + case _ => "" + } + Seq( execId, hostPort, rddBlocks, - "%s/%s".format(memUsed, maxMem), - diskUsed + "%s / %s".format(memUsed, maxMem), + diskUsed, + "%s / %s".format(completedTasks, totalTasks) + failedTasks ) } + + private[spark] class ExecutorsListener extends SparkListener with Logging { + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToTaskInfos = + HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val eid = taskEnd.taskMetrics.executorId + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + logInfo("Executor %s has %s failed tasks.".format(eid, executorToTasksFailed(eid))) + (Some(e), e.metrics) + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + logInfo("Executor %s has %s completed tasks.".format(eid, executorToTasksComplete(eid))) + (None, Some(taskEnd.taskMetrics)) + } + val taskList = executorToTaskInfos.getOrElse( + eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskEnd.taskInfo, metrics, failureInfo)) + executorToTaskInfos(eid) = taskList + } + } } \ No newline at end of file From 2eea974795dfa2bb79e66496454f36cb499065b0 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 15:15:54 -0700 Subject: [PATCH 106/221] Executors UI now calls executor ID from TaskInfo instead of TaskMetrics --- core/src/main/scala/spark/executor/Executor.scala | 1 - core/src/main/scala/spark/executor/TaskMetrics.scala | 5 ----- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 4 +--- 3 files changed, 1 insertion(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 82e5f5a015..2e81151882 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -126,7 +126,6 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => m.hostname = Utils.localHostName - m.executorId = executorId m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index e14dc57501..3151627839 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -23,11 +23,6 @@ class TaskMetrics extends Serializable { */ var hostname: String = _ - /** - * Executor's ID the task runs on - */ - var executorId: String = _ - /** * Time taken on the executor to deserialize this task */ diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index a981c680d2..40bee325b2 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -111,16 +111,14 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val eid = taskEnd.taskMetrics.executorId + val eid = taskEnd.taskInfo.executorId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - logInfo("Executor %s has %s failed tasks.".format(eid, executorToTasksFailed(eid))) (Some(e), e.metrics) case _ => executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - logInfo("Executor %s has %s completed tasks.".format(eid, executorToTasksComplete(eid))) (None, Some(taskEnd.taskMetrics)) } val taskList = executorToTaskInfos.getOrElse( From 8e38e7723230c1ca9564ddb79087fd1c4326b351 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 22 Jul 2013 16:05:32 -0700 Subject: [PATCH 107/221] Fix a test that was using an outdated config setting --- core/src/test/scala/spark/ShuffleSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 3a56c26bef..752e4b85e6 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -33,9 +33,9 @@ import spark.rdd.ShuffledRDD import spark.SparkContext._ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { - test("groupByKey with compression") { + test("groupByKey without compression") { try { - System.setProperty("spark.shuffle.compress", "true") + System.setProperty("spark.shuffle.compress", "false") sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) val groups = pairs.groupByKey(4).collect() @@ -45,7 +45,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val valuesFor2 = groups.find(_._1 == 2).get._2 assert(valuesFor2.toList.sorted === List(1)) } finally { - System.setProperty("spark.blockManager.compress", "false") + System.setProperty("spark.shuffle.compress", "true") } } From c83680434bfd6c241ed2e66f2e8704d3b93f752f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 17 Jul 2013 22:59:00 -0700 Subject: [PATCH 108/221] Add JavaAPICompletenessChecker. This is used to find methods in the Scala API that need to be ported to the Java API. To use it: ./run spark.tools.JavaAPICompletenessChecker Conflicts: project/SparkBuild.scala run run2.cmd --- bin/compute-classpath.cmd | 2 + bin/compute-classpath.sh | 2 + pom.xml | 1 + project/SparkBuild.scala | 8 +- tools/pom.xml | 63 +++ .../tools/JavaAPICompletenessChecker.scala | 359 ++++++++++++++++++ 6 files changed, 434 insertions(+), 1 deletion(-) create mode 100644 tools/pom.xml create mode 100644 tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 835d1af794..eb836b0ffd 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -33,6 +33,7 @@ set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel set MLLIB_DIR=%FWDIR%mllib +set TOOLS_DIR=%FWDIR%tools set STREAMING_DIR=%FWDIR%streaming set PYSPARK_DIR=%FWDIR%python @@ -48,6 +49,7 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes set CLASSPATH=%CLASSPATH%;%MLLIB_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%TOOLS_DIR%\target\scala-%SCALA_VERSION%\classes 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 diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index e6fdc5306e..eb270a5428 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -36,6 +36,7 @@ REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" MLLIB_DIR="$FWDIR/mllib" +TOOLS_DIR="$FWDIR/tools" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -70,6 +71,7 @@ function dev_classpath { fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$TOOLS_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH="$CLASSPATH:$jar" done diff --git a/pom.xml b/pom.xml index 4b48072c6e..eb7bd7e9df 100644 --- a/pom.xml +++ b/pom.xml @@ -58,6 +58,7 @@ core bagel examples + tools streaming repl repl-bin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2f2cbf646a..f3f67b57c8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -40,7 +40,7 @@ object SparkBuild extends Build { //val HADOOP_MAJOR_VERSION = "2" //val HADOOP_YARN = true - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib, tools) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -48,6 +48,8 @@ object SparkBuild extends Build { lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) + lazy val tools = Project("tools", file("tools"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) + lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) @@ -233,6 +235,10 @@ object SparkBuild extends Build { ) ) + def toolsSettings = sharedSettings ++ Seq( + name := "spark-tools" + ) + def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") def mllibSettings = sharedSettings ++ Seq( diff --git a/tools/pom.xml b/tools/pom.xml new file mode 100644 index 0000000000..3dfba5808c --- /dev/null +++ b/tools/pom.xml @@ -0,0 +1,63 @@ + + + + 4.0.0 + + org.spark-project + spark-parent + 0.7.4-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-tools_2.9.3 + jar + Spark Project Tools + http://spark-project.org/ + + + + org.spark-project + spark-core_2.9.3 + ${project.version} + + + org.spark-project + spark-streaming_2.9.3 + ${project.version} + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.apache.maven.plugins + maven-source-plugin + + + org.codehaus.mojo + build-helper-maven-plugin + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala new file mode 100644 index 0000000000..89ef7236f4 --- /dev/null +++ b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.tools + +import spark._ +import java.lang.reflect.Method +import scala.collection.mutable.ArrayBuffer +import spark.api.java._ +import spark.streaming.{PairDStreamFunctions, DStream, StreamingContext} +import spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext} +import scala.Tuple2 + + +private[spark] abstract class SparkType(val name: String) + +private[spark] case class BaseType(override val name: String) extends SparkType(name) { + override def toString: String = { + name + } +} + +private[spark] +case class ParameterizedType(override val name: String, + parameters: Seq[SparkType], + typebounds: String = "") extends SparkType(name) { + override def toString: String = { + if (typebounds != "") { + typebounds + " " + name + "<" + parameters.mkString(", ") + ">" + } else { + name + "<" + parameters.mkString(", ") + ">" + } + } +} + +private[spark] +case class SparkMethod(name: String, returnType: SparkType, parameters: Seq[SparkType]) { + override def toString: String = { + returnType + " " + name + "(" + parameters.mkString(", ") + ")" + } +} + +/** + * A tool for identifying methods that need to be ported from Scala to the Java API. + * + * It uses reflection to find methods in the Scala API and rewrites those methods' signatures + * into appropriate Java equivalents. If those equivalent methods have not been implemented in + * the Java API, they are printed. + */ +object JavaAPICompletenessChecker { + + private def parseType(typeStr: String): SparkType = { + if (!typeStr.contains("<")) { + // Base types might begin with "class" or "interface", so we have to strip that off: + BaseType(typeStr.trim.split(" ").last) + } else if (typeStr.endsWith("[]")) { + ParameterizedType("Array", Seq(parseType(typeStr.stripSuffix("[]")))) + } else { + val parts = typeStr.split("<", 2) + val name = parts(0).trim + assert (parts(1).last == '>') + val parameters = parts(1).dropRight(1) + ParameterizedType(name, parseTypeList(parameters)) + } + } + + private def parseTypeList(typeStr: String): Seq[SparkType] = { + val types: ArrayBuffer[SparkType] = new ArrayBuffer[SparkType] + var stack = 0 + var token: StringBuffer = new StringBuffer() + for (c <- typeStr.trim) { + if (c == ',' && stack == 0) { + types += parseType(token.toString) + token = new StringBuffer() + } else if (c == ' ' && stack != 0) { + // continue + } else { + if (c == '<') { + stack += 1 + } else if (c == '>') { + stack -= 1 + } + token.append(c) + } + } + assert (stack == 0) + if (token.toString != "") { + types += parseType(token.toString) + } + types.toSeq + } + + private def parseReturnType(typeStr: String): SparkType = { + if (typeStr(0) == '<') { + val parts = typeStr.drop(0).split(">", 2) + val parsed = parseType(parts(1)).asInstanceOf[ParameterizedType] + ParameterizedType(parsed.name, parsed.parameters, parts(0)) + } else { + parseType(typeStr) + } + } + + private def toSparkMethod(method: Method): SparkMethod = { + val returnType = parseReturnType(method.getGenericReturnType.toString) + val name = method.getName + val parameters = method.getGenericParameterTypes.map(t => parseType(t.toString)) + SparkMethod(name, returnType, parameters) + } + + private def toJavaType(scalaType: SparkType): SparkType = { + val renameSubstitutions = Map( + "scala.collection.Map" -> "java.util.Map", + // TODO: the JavaStreamingContext API accepts Array arguments + // instead of Lists, so this isn't a trivial translation / sub: + "scala.collection.Seq" -> "java.util.List", + "scala.Function2" -> "spark.api.java.function.Function2", + "scala.collection.Iterator" -> "java.util.Iterator", + "scala.collection.mutable.Queue" -> "java.util.Queue", + "double" -> "java.lang.Double" + ) + // Keep applying the substitutions until we've reached a fixedpoint. + def applySubs(scalaType: SparkType): SparkType = { + scalaType match { + case ParameterizedType(name, parameters, typebounds) => + name match { + case "spark.RDD" => + if (parameters(0).name == classOf[Tuple2[_, _]].getName) { + val tupleParams = + parameters(0).asInstanceOf[ParameterizedType].parameters.map(toJavaType) + ParameterizedType(classOf[JavaPairRDD[_, _]].getName, tupleParams) + } else { + ParameterizedType(classOf[JavaRDD[_]].getName, parameters.map(toJavaType)) + } + case "spark.streaming.DStream" => + if (parameters(0).name == classOf[Tuple2[_, _]].getName) { + val tupleParams = + parameters(0).asInstanceOf[ParameterizedType].parameters.map(toJavaType) + ParameterizedType("spark.streaming.api.java.JavaPairDStream", tupleParams) + } else { + ParameterizedType("spark.streaming.api.java.JavaDStream", + parameters.map(toJavaType)) + } + // TODO: Spark Streaming uses Guava's Optional in place of Option, leading to some + // false-positives here: + case "scala.Option" => + toJavaType(parameters(0)) + case "scala.Function1" => + val firstParamName = parameters.last.name + if (firstParamName.startsWith("scala.collection.Traversable") || + firstParamName.startsWith("scala.collection.Iterator")) { + ParameterizedType("spark.api.java.function.FlatMapFunction", + Seq(parameters(0), + parameters.last.asInstanceOf[ParameterizedType].parameters(0)).map(toJavaType)) + } else if (firstParamName == "scala.runtime.BoxedUnit") { + ParameterizedType("spark.api.java.function.VoidFunction", + parameters.dropRight(1).map(toJavaType)) + } else { + ParameterizedType("spark.api.java.function.Function", parameters.map(toJavaType)) + } + case _ => + ParameterizedType(renameSubstitutions.getOrElse(name, name), + parameters.map(toJavaType)) + } + case BaseType(name) => + if (renameSubstitutions.contains(name)) { + BaseType(renameSubstitutions(name)) + } else { + scalaType + } + } + } + var oldType = scalaType + var newType = applySubs(scalaType) + while (oldType != newType) { + oldType = newType + newType = applySubs(scalaType) + } + newType + } + + private def toJavaMethod(method: SparkMethod): SparkMethod = { + val params = method.parameters + .filterNot(_.name == "scala.reflect.ClassManifest").map(toJavaType) + SparkMethod(method.name, toJavaType(method.returnType), params) + } + + private def isExcludedByName(method: Method): Boolean = { + val name = method.getDeclaringClass.getName + "." + method.getName + // Scala methods that are declared as private[mypackage] become public in the resulting + // Java bytecode. As a result, we need to manually exclude those methods here. + // This list also includes a few methods that are only used by the web UI or other + // internal Spark components. + val excludedNames = Seq( + "spark.RDD.origin", + "spark.RDD.elementClassManifest", + "spark.RDD.checkpointData", + "spark.RDD.partitioner", + "spark.RDD.partitions", + "spark.RDD.firstParent", + "spark.RDD.doCheckpoint", + "spark.RDD.markCheckpointed", + "spark.RDD.clearDependencies", + "spark.RDD.getDependencies", + "spark.RDD.getPartitions", + "spark.RDD.dependencies", + "spark.RDD.getPreferredLocations", + "spark.RDD.collectPartitions", + "spark.RDD.computeOrReadCheckpoint", + "spark.PairRDDFunctions.getKeyClass", + "spark.PairRDDFunctions.getValueClass", + "spark.SparkContext.stringToText", + "spark.SparkContext.makeRDD", + "spark.SparkContext.runJob", + "spark.SparkContext.runApproximateJob", + "spark.SparkContext.clean", + "spark.SparkContext.metadataCleaner", + "spark.SparkContext.ui", + "spark.SparkContext.newShuffleId", + "spark.SparkContext.newRddId", + "spark.SparkContext.cleanup", + "spark.SparkContext.receiverJobThread", + "spark.SparkContext.getRDDStorageInfo", + "spark.SparkContext.addedFiles", + "spark.SparkContext.addedJars", + "spark.SparkContext.persistentRdds", + "spark.SparkContext.executorEnvs", + "spark.SparkContext.checkpointDir", + "spark.SparkContext.getSparkHome", + "spark.SparkContext.executorMemoryRequested", + "spark.SparkContext.getExecutorStorageStatus", + "spark.streaming.DStream.generatedRDDs", + "spark.streaming.DStream.zeroTime", + "spark.streaming.DStream.rememberDuration", + "spark.streaming.DStream.storageLevel", + "spark.streaming.DStream.mustCheckpoint", + "spark.streaming.DStream.checkpointDuration", + "spark.streaming.DStream.checkpointData", + "spark.streaming.DStream.graph", + "spark.streaming.DStream.isInitialized", + "spark.streaming.DStream.parentRememberDuration", + "spark.streaming.DStream.initialize", + "spark.streaming.DStream.validate", + "spark.streaming.DStream.setContext", + "spark.streaming.DStream.setGraph", + "spark.streaming.DStream.remember", + "spark.streaming.DStream.getOrCompute", + "spark.streaming.DStream.generateJob", + "spark.streaming.DStream.clearOldMetadata", + "spark.streaming.DStream.addMetadata", + "spark.streaming.DStream.updateCheckpointData", + "spark.streaming.DStream.restoreCheckpointData", + "spark.streaming.DStream.isTimeValid", + "spark.streaming.StreamingContext.nextNetworkInputStreamId", + "spark.streaming.StreamingContext.networkInputTracker", + "spark.streaming.StreamingContext.checkpointDir", + "spark.streaming.StreamingContext.checkpointDuration", + "spark.streaming.StreamingContext.receiverJobThread", + "spark.streaming.StreamingContext.scheduler", + "spark.streaming.StreamingContext.initialCheckpoint", + "spark.streaming.StreamingContext.getNewNetworkStreamId", + "spark.streaming.StreamingContext.validate", + "spark.streaming.StreamingContext.createNewSparkContext", + "spark.streaming.StreamingContext.rddToFileName", + "spark.streaming.StreamingContext.getSparkCheckpointDir", + "spark.streaming.StreamingContext.env", + "spark.streaming.StreamingContext.graph", + "spark.streaming.StreamingContext.isCheckpointPresent" + ) + val excludedPatterns = Seq( + """^spark\.SparkContext\..*To.*Functions""", + """^spark\.SparkContext\..*WritableConverter""", + """^spark\.SparkContext\..*To.*Writable""" + ).map(_.r) + lazy val excludedByPattern = + !excludedPatterns.map(_.findFirstIn(name)).filter(_.isDefined).isEmpty + name.contains("$") || excludedNames.contains(name) || excludedByPattern + } + + private def isExcludedByAnnotation(method: Method): Boolean = { + method.getAnnotation(classOf[ExcludeFromJavaAPI]) != null + } + + private def isExcludedByInterface(method: Method): Boolean = { + val excludedInterfaces = + Set("spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") + def toComparisionKey(method: Method) = + (method.getReturnType, method.getName, method.getGenericReturnType) + val interfaces = method.getDeclaringClass.getInterfaces.filter { i => + excludedInterfaces.contains(i.getName) + } + val excludedMethods = interfaces.flatMap(_.getMethods.map(toComparisionKey)) + excludedMethods.contains(toComparisionKey(method)) + } + + private def printMissingMethods(scalaClass: Class[_], javaClass: Class[_]) { + val methods = scalaClass.getMethods + .filterNot(_.isAccessible) + .filterNot(isExcludedByAnnotation) + .filterNot(isExcludedByName) + .filterNot(isExcludedByInterface) + val javaEquivalents = methods.map(m => toJavaMethod(toSparkMethod(m))).toSet + + val javaMethods = javaClass.getMethods.map(toSparkMethod).toSet + + val missingMethods = javaEquivalents -- javaMethods + + for (method <- missingMethods) { + println(method) + } + } + + def main(args: Array[String]) { + println("Missing RDD methods") + printMissingMethods(classOf[RDD[_]], classOf[JavaRDD[_]]) + println() + + println("Missing PairRDD methods") + printMissingMethods(classOf[PairRDDFunctions[_, _]], classOf[JavaPairRDD[_, _]]) + println() + + println("Missing DoubleRDD methods") + printMissingMethods(classOf[DoubleRDDFunctions], classOf[JavaDoubleRDD]) + println() + + println("Missing OrderedRDD methods") + printMissingMethods(classOf[OrderedRDDFunctions[_, _]], classOf[JavaPairRDD[_, _]]) + println() + + println("Missing SparkContext methods") + printMissingMethods(classOf[SparkContext], classOf[JavaSparkContext]) + println() + + println("Missing StreamingContext methods") + printMissingMethods(classOf[StreamingContext], classOf[JavaStreamingContext]) + println() + + println("Missing DStream methods") + printMissingMethods(classOf[DStream[_]], classOf[JavaDStream[_]]) + println() + + println("Missing PairDStream methods") + printMissingMethods(classOf[PairDStreamFunctions[_, _]], classOf[JavaPairDStream[_, _]]) + println() + } +} From e17e1b388eaddea39a7272bae201fd3d0060c821 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 20 Jul 2013 22:04:30 -0700 Subject: [PATCH 109/221] Remove annotation code that broke build. --- .../main/scala/spark/tools/JavaAPICompletenessChecker.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala index 89ef7236f4..3a55f50812 100644 --- a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala @@ -290,10 +290,6 @@ object JavaAPICompletenessChecker { name.contains("$") || excludedNames.contains(name) || excludedByPattern } - private def isExcludedByAnnotation(method: Method): Boolean = { - method.getAnnotation(classOf[ExcludeFromJavaAPI]) != null - } - private def isExcludedByInterface(method: Method): Boolean = { val excludedInterfaces = Set("spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") @@ -309,7 +305,6 @@ object JavaAPICompletenessChecker { private def printMissingMethods(scalaClass: Class[_], javaClass: Class[_]) { val methods = scalaClass.getMethods .filterNot(_.isAccessible) - .filterNot(isExcludedByAnnotation) .filterNot(isExcludedByName) .filterNot(isExcludedByInterface) val javaEquivalents = methods.map(m => toJavaMethod(toSparkMethod(m))).toSet From 872c97ad829ba20e866c4e45054e7d2d05b02042 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 16:54:37 -0700 Subject: [PATCH 110/221] Split task columns, memory columns sort by numeric value --- .../scala/spark/ui/exec/ExecutorsUI.scala | 36 +++++++++++-------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 40bee325b2..20ea54d6a6 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -45,15 +45,21 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { .reduceOption(_+_).getOrElse(0L) val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", - "Tasks: Complete/Total") + "Failed tasks", "Complete tasks", "Total tasks") def execRow(kv: Seq[String]) = {kv(0)} {kv(1)} {kv(2)} - {kv(3)} - {kv(4)} - {kv(5)} + + {Utils.memoryBytesToString(kv(3).toLong)} / {Utils.memoryBytesToString(kv(4).toLong)} + + + {Utils.memoryBytesToString(kv(5).toLong)} + + {kv(6)} + {kv(7)} + {kv(8)} val execInfo = for (b <- 0 until storageStatusList.size) @@ -83,24 +89,24 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { def getExecInfo(a: Int): Seq[String] = { val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) - val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) - val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString - val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0) - val totalTasks = listener.executorToTaskInfos(a.toString).size - val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0) match { - case f if f > 0 => " (%s failed)".format(f) - case _ => "" - } + val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString + val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString + val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString + val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString + val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString + val totalTasks = listener.executorToTaskInfos(a.toString).size.toString Seq( execId, hostPort, rddBlocks, - "%s / %s".format(memUsed, maxMem), + memUsed, + maxMem, diskUsed, - "%s / %s".format(completedTasks, totalTasks) + failedTasks + failedTasks, + completedTasks, + totalTasks ) } From 2c2bfbe294c0082520c80a01562a2dbeeba63b7a Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 01:36:44 -0700 Subject: [PATCH 111/221] Add toMap method to TimeStampedHashMap and use it --- core/src/main/scala/spark/SparkContext.scala | 2 +- core/src/main/scala/spark/util/TimeStampedHashMap.scala | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1b46665d2c..0fb7dfa810 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -550,7 +550,7 @@ class SparkContext( * Returns an immutable map of RDDs that have marked themselves as cached via cache() call. * Note that this does not necessarily mean the caching or computation was successful. */ - def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.asInstanceOf[Map[Int, RDD[_]]] + def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.toMap def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index cc7909194a..07772a0afb 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -20,6 +20,7 @@ package spark.util import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Map +import scala.collection.immutable import spark.scheduler.MapStatus /** @@ -99,6 +100,8 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging { } } + def toMap: immutable.Map[A, B] = iterator.toMap + /** * Removes old key-value pairs that have timestamp earlier than `threshTime` */ From 4830e225624091fa836012651420cf2b5b97dcca Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 09:50:13 -0700 Subject: [PATCH 112/221] Rename method per rxin feedback --- core/src/main/scala/spark/SparkContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 0fb7dfa810..24ba605646 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -547,10 +547,10 @@ class SparkContext( } /** - * Returns an immutable map of RDDs that have marked themselves as cached via cache() call. + * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call. * Note that this does not necessarily mean the caching or computation was successful. */ - def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.toMap + def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos From efd6418c1b99c1ecc2b0a4c72e6430eea4d86260 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 10:40:41 -0700 Subject: [PATCH 113/221] Move getPersistentRDDs testing to a new Suite --- core/src/test/scala/spark/RDDSuite.scala | 6 -- .../scala/spark/SparkContextInfoSuite.scala | 60 +++++++++++++++++++ 2 files changed, 60 insertions(+), 6 deletions(-) create mode 100644 core/src/test/scala/spark/SparkContextInfoSuite.scala diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index ff2dcd72d8..cbddf4e523 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -90,19 +90,15 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("basic caching") { - val origCachedRdds = sc.getCachedRDDs.size val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) - // Should only result in one cached RDD - assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("caching with failures") { val onlySplit = new Partition { override def index: Int = 0 } var shouldFail = true - val origCachedRdds = sc.getCachedRDDs.size val rdd = new RDD[Int](sc, Nil) { override def getPartitions: Array[Partition] = Array(onlySplit) override val getDependencies = List[Dependency[_]]() @@ -114,14 +110,12 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } }.cache() - assert(sc.getCachedRDDs.size === origCachedRdds + 1) val thrown = intercept[Exception]{ rdd.collect() } assert(thrown.getMessage.contains("injected failure")) shouldFail = false assert(rdd.collect().toList === List(1, 2, 3, 4)) - assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("empty RDD") { diff --git a/core/src/test/scala/spark/SparkContextInfoSuite.scala b/core/src/test/scala/spark/SparkContextInfoSuite.scala new file mode 100644 index 0000000000..6d50bf5e1b --- /dev/null +++ b/core/src/test/scala/spark/SparkContextInfoSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark + +import org.scalatest.FunSuite +import spark.SparkContext._ + +class SparkContextInfoSuite extends FunSuite with LocalSparkContext { + test("getPersistentRDDs only returns RDDs that are marked as cached") { + sc = new SparkContext("local", "test") + assert(sc.getPersistentRDDs.isEmpty === true) + + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) + assert(sc.getPersistentRDDs.isEmpty === true) + + rdd.cache() + assert(sc.getPersistentRDDs.size === 1) + assert(sc.getPersistentRDDs.values.head === rdd) + } + + test("getPersistentRDDs returns an immutable map") { + sc = new SparkContext("local", "test") + val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + + val myRdds = sc.getPersistentRDDs + assert(myRdds.size === 1) + assert(myRdds.values.head === rdd1) + + val rdd2 = sc.makeRDD(Array(5, 6, 7, 8), 1).cache() + + // getPersistentRDDs should have 2 RDDs, but myRdds should not change + assert(sc.getPersistentRDDs.size === 2) + assert(myRdds.size === 1) + } + + test("getRDDStorageInfo only reports on RDDs that actually persist data") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + + assert(sc.getRDDStorageInfo.size === 0) + + rdd.collect() + assert(sc.getRDDStorageInfo.size === 1) + } +} \ No newline at end of file From 87a9dd898ff51fd110799edae087d59f6b714211 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 12:13:27 -0700 Subject: [PATCH 114/221] Made RegressionModel serializable and added unit tests to make sure predict methods would work. --- .../mllib/optimization/GradientDescent.scala | 6 ++-- .../spark/mllib/optimization/Updater.scala | 6 ++-- .../mllib/regression/LogisticRegression.scala | 6 +++- .../spark/mllib/regression/Regression.scala | 2 +- .../mllib/regression/RidgeRegression.scala | 5 ++- .../regression/LogisticRegressionSuite.scala | 33 +++++++++++++++---- 6 files changed, 42 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 4c996c0903..185a2a24f6 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -39,9 +39,9 @@ object GradientDescent { * @param miniBatchFraction - fraction of the input data set that should be used for * one iteration of SGD. Default value 1.0. * - * @return weights - Column matrix containing weights for every feature. - * @return stochasticLossHistory - Array containing the stochastic loss computed for - * every iteration. + * @return A tuple containing two elements. The first element is a column matrix containing + * weights for every feature, and the second element is an array containing the stochastic + * loss computed for every iteration. */ def runMiniBatchSGD( data: RDD[(Double, Array[Double])], diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index b864fd4634..18cb5f3a95 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -23,13 +23,13 @@ abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize and iteration number. * - * @param weightsOld - Column matrix of size nx1 where n is the number of features. + * @param weightsOlds - Column matrix of size nx1 where n is the number of features. * @param gradient - Column matrix of size nx1 where n is the number of features. * @param stepSize - step size across iterations * @param iter - Iteration number * - * @return weightsNew - Column matrix containing updated weights - * @return reg_val - regularization value + * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, + * and the second element is the regularization value. */ def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): (DoubleMatrix, Double) diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index 711e205c39..4b22546017 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -36,8 +36,12 @@ class LogisticRegressionModel( private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) override def predict(testData: spark.RDD[Array[Double]]) = { + // A small optimization to avoid serializing the entire model. Only the weightsMatrix + // and intercept is needed. + val localWeights = weightsMatrix + val localIntercept = intercept testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(weightsMatrix).get(0) + this.intercept + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept 1.0/ (1.0 + math.exp(margin * -1)) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala index 645204ddf3..b845ba1a89 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Regression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Regression.scala @@ -19,7 +19,7 @@ package spark.mllib.regression import spark.RDD -trait RegressionModel { +trait RegressionModel extends Serializable { /** * Predict values for the given data set using the model trained. * diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index f724edd732..6ba141e8fb 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -37,8 +37,11 @@ class RidgeRegressionModel( extends RegressionModel { override def predict(testData: RDD[Array[Double]]): RDD[Double] = { + // A small optimization to avoid serializing the entire model. + val localIntercept = this.intercept + val localWeights = this.weights testData.map { x => - (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + (new DoubleMatrix(1, x.length, x:_*).mmul(localWeights)).get(0) + localIntercept } } diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 47191d9a5a..6a8098b59d 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { @@ -51,15 +50,24 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => + val y: Seq[Double] = (0 until nPoints).map { i => val yVal = offset + scale * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))) + val testData = (0 until nPoints).map(i => (y(i), Array(x1(i)))) testData } + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + val offPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(offPredictions < input.length / 5) + } + // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("logistic regression") { val nPoints = 10000 @@ -70,14 +78,20 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(10.0) - .setNumIterations(20) + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(20) val model = lr.train(testRDD) + // Test the weights val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + // Test prediction on RDD. + validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + + // Test prediction on Array. + validatePrediction(testData.map(row => model.predict(row._2)), testData) } test("logistic regression with initial weights") { @@ -94,13 +108,18 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { testRDD.cache() // Use half as many iterations as the previous test. - val lr = new LogisticRegression().setStepSize(10.0) - .setNumIterations(10) + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(10) val model = lr.train(testRDD, initialWeights) val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + // Test prediction on RDD. + validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + + // Test prediction on Array. + validatePrediction(testData.map(row => model.predict(row._2)), testData) } } From 2210e8ccf8d77f65442a344c4eae39e000fba927 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 12:52:15 -0700 Subject: [PATCH 115/221] Use a different validation dataset for Logistic Regression prediction testing. --- .../regression/LogisticRegressionSuite.scala | 29 +++++++++++-------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 6a8098b59d..0a99b78cf8 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -35,10 +35,11 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { // Generate input of the form Y = logistic(offset + scale*X) def generateLogisticInput( - offset: Double, - scale: Double, - nPoints: Int) : Seq[(Double, Array[Double])] = { - val rnd = new Random(42) + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) @@ -60,12 +61,12 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { } def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { - val offPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => // A prediction is off if the prediction is more than 0.5 away from expected value. math.abs(prediction - expected) > 0.5 }.size // At least 80% of the predictions should be on. - assert(offPredictions < input.length / 5) + assert(numOffPredictions < input.length / 5) } // Test if we can correctly learn A, B where Y = logistic(A + B*X) @@ -74,7 +75,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val A = 2.0 val B = -1.5 - val testData = generateLogisticInput(A, B, nPoints) + val testData = generateLogisticInput(A, B, nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() @@ -87,11 +88,13 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. - validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) // Test prediction on Array. - validatePrediction(testData.map(row => model.predict(row._2)), testData) + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } test("logistic regression with initial weights") { @@ -99,7 +102,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val A = 2.0 val B = -1.5 - val testData = generateLogisticInput(A, B, nPoints) + val testData = generateLogisticInput(A, B, nPoints, 42) val initialB = -1.0 val initialWeights = Array(initialB) @@ -116,10 +119,12 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. - validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) // Test prediction on Array. - validatePrediction(testData.map(row => model.predict(row._2)), testData) + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } From 0200801a55b580c7504687e3476b7a71c7699001 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 13:35:43 -0700 Subject: [PATCH 116/221] Tracks task start events and shows number of active tasks on Executor UI --- .../scala/spark/scheduler/DAGScheduler.scala | 8 ++++++++ .../spark/scheduler/DAGSchedulerEvent.scala | 2 ++ .../main/scala/spark/scheduler/JobLogger.scala | 16 +++++++++++++++- .../scala/spark/scheduler/SparkListener.scala | 9 ++++++++- .../spark/scheduler/TaskSchedulerListener.scala | 3 +++ .../cluster/ClusterTaskSetManager.scala | 6 ++++++ .../scheduler/local/LocalTaskSetManager.scala | 5 +++++ .../main/scala/spark/ui/exec/ExecutorsUI.scala | 12 +++++++++++- 8 files changed, 58 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 29e879aa42..b02bf8f4bf 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -52,6 +52,11 @@ class DAGScheduler( } taskSched.setListener(this) + //Called by TaskScheduler to report task's starting. + override def taskStarted(task: Task[_], taskInfo: TaskInfo) { + eventQueue.put(BeginEvent(task, taskInfo)) + } + // Called by TaskScheduler to report task completions or failures. override def taskEnded( task: Task[_], @@ -343,6 +348,9 @@ class DAGScheduler( case ExecutorLost(execId) => handleExecutorLost(execId) + case begin: BeginEvent => + sparkListeners.foreach(_.onTaskStart(SparkListenerTaskStart(begin.task, begin.taskInfo))) + case completion: CompletionEvent => sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion.task, completion.reason, completion.taskInfo, completion.taskMetrics))) diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 506c87f65b..3b4ee6287a 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -43,6 +43,8 @@ private[spark] case class JobSubmitted( properties: Properties = null) extends DAGSchedulerEvent +private[spark] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent + private[spark] case class CompletionEvent( task: Task[_], reason: TaskEndReason, diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 85b5ddd4a8..f7565b8c57 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -68,6 +68,8 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { processStageCompletedEvent(stageInfo) case SparkListenerJobEnd(job, result) => processJobEndEvent(job, result) + case SparkListenerTaskStart(task, taskInfo) => + processTaskStartEvent(task, taskInfo) case SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics) => processTaskEndEvent(task, reason, taskInfo, taskMetrics) case _ => @@ -252,7 +254,19 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { stageInfo.stage.id + " STATUS=COMPLETED") } - + + override def onTaskStart(taskStart: SparkListenerTaskStart) { + eventQueue.put(taskStart) + } + + protected def processTaskStartEvent(task: Task[_], taskInfo: TaskInfo) { + var taskStatus = "" + task match { + case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" + case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" + } + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { eventQueue.put(taskEnd) } diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 4fb1c5d42d..4eb7e4e6a5 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -29,6 +29,8 @@ case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends Spar case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents +case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents + case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvents @@ -48,7 +50,12 @@ trait SparkListener { * Called when a stage is submitted */ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } - + + /** + * Called when a task starts + */ + def onTaskStart(taskEnd: SparkListenerTaskStart) { } + /** * Called when a task ends */ diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index 245e7ccb52..2cdeb1c8c0 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -27,6 +27,9 @@ import spark.executor.TaskMetrics * Interface for getting events back from the TaskScheduler. */ private[spark] trait TaskSchedulerListener { + // A task has started. + def taskStarted(task: Task[_], taskInfo: TaskInfo) + // A task has finished or failed. def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 3d06520675..14e87af653 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -496,6 +496,8 @@ private[spark] class ClusterTaskSetManager( logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) val taskName = "task %s:%d".format(taskSet.id, index) + if (taskAttempts(index).size == 1) + taskStarted(task,info) return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) } case _ => @@ -518,6 +520,10 @@ private[spark] class ClusterTaskSetManager( } } + def taskStarted(task: Task[_], info: TaskInfo) { + sched.listener.taskStarted(task, info) + } + def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { val info = taskInfos(tid) if (info.failed) { diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index e662ad6709..b500451990 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -117,6 +117,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val taskName = "task %s:%d".format(taskSet.id, index) copiesRunning(index) += 1 increaseRunningTasks(1) + taskStarted(task, info) return Some(new TaskDescription(taskId, null, taskName, bytes)) case None => {} } @@ -146,6 +147,10 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } } + def taskStarted(task: Task[_], info: TaskInfo) { + sched.listener.taskStarted(task, info) + } + def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { val info = taskInfos(tid) val index = info.index diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 20ea54d6a6..9ac33326c0 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -45,7 +45,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { .reduceOption(_+_).getOrElse(0L) val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", - "Failed tasks", "Complete tasks", "Total tasks") + "Active tasks", "Failed tasks", "Complete tasks", "Total tasks") def execRow(kv: Seq[String]) = {kv(0)} @@ -60,6 +60,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {kv(6)} {kv(7)} {kv(8)} + {kv(9)} val execInfo = for (b <- 0 until storageStatusList.size) @@ -93,6 +94,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString + val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, 0).toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString @@ -104,6 +106,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { memUsed, maxMem, diskUsed, + activeTasks, failedTasks, completedTasks, totalTasks @@ -111,13 +114,20 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { + val executorToTasksActive = HashMap[String, Int]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + override def onTaskStart(taskStart: SparkListenerTaskStart) { + val eid = taskStart.taskInfo.executorId + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) - 1 val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => From 9f2dbb2a7ca9c1cf878cf96df7122b05d95e799b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:10:09 -0700 Subject: [PATCH 117/221] Adds/removes active tasks only once --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 9ac33326c0..bb2b003486 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -94,7 +94,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, 0).toString + val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, Seq[Long]()).size.toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString @@ -114,7 +114,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, Int]() + val executorToTasksActive = HashMap[String, Seq[Long]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = @@ -122,12 +122,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()) :+ taskStart.taskInfo.taskId } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) - 1 + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()).filterNot(_ == taskEnd.taskInfo.taskId) val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => From 101b8cc78a1fb49332265d7a7e4bf4ec6a22dd42 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:20 -0700 Subject: [PATCH 118/221] SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure. --- .../scala/spark/scheduler/DAGScheduler.scala | 29 +++++++++++++------ core/src/test/scala/spark/FailureSuite.scala | 29 ++++++++++++++----- 2 files changed, 42 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 29e879aa42..5fcd807aff 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -17,19 +17,17 @@ package spark.scheduler -import cluster.TaskInfo -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.LinkedBlockingQueue -import java.util.concurrent.TimeUnit +import java.io.NotSerializableException import java.util.Properties +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ import spark.executor.TaskMetrics -import spark.partial.ApproximateActionListener -import spark.partial.ApproximateEvaluator -import spark.partial.PartialResult +import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} +import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -258,7 +256,8 @@ class DAGScheduler( assert(partitions.size > 0) val waiter = new JobWaiter(partitions.size, resultHandler) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) + val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, + properties) return (toSubmit, waiter) } @@ -283,7 +282,7 @@ class DAGScheduler( "Total number of partitions: " + maxPartitions) } - val (toSubmit, waiter) = prepareJob( + val (toSubmit: JobSubmitted, waiter: JobWaiter[_]) = prepareJob( finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) waiter.awaitResult() match { @@ -466,6 +465,18 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") + + // Preemptively serialize the stage RDD to make sure the tasks for this stage will be + // serializable. We are catching this exception here because it would be fairly hard to + // catch the non-serializable exception down the road, where we have several different + // implementations for local scheduler and cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) + } catch { + case e: NotSerializableException => abortStage(stage, e.toString) + return + } + if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index 6c847b8fef..c3c52f9118 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -18,9 +18,6 @@ package spark import org.scalatest.FunSuite -import org.scalatest.prop.Checkers - -import scala.collection.mutable.ArrayBuffer import SparkContext._ @@ -40,7 +37,7 @@ object FailureSuiteState { } class FailureSuite extends FunSuite with LocalSparkContext { - + // Run a 3-task map job in which task 1 deterministically fails once, and check // whether the job completes successfully and we ran 4 tasks in total. test("failure in a single-stage job") { @@ -66,7 +63,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure in a two-stage job") { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map { - case (k, v) => + case (k, v) => FailureSuiteState.synchronized { FailureSuiteState.tasksRun += 1 if (k == 1 && FailureSuiteState.tasksFailed == 0) { @@ -87,15 +84,33 @@ class FailureSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => new NonSerializable) - val thrown = intercept[spark.SparkException] { + val thrown = intercept[SparkException] { results.collect() } - assert(thrown.getClass === classOf[spark.SparkException]) + assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) FailureSuiteState.clear() } + test("failure because task closure is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + val thrown = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => a).count() + } + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("NotSerializableException")) + + val thrown1 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() + } + assert(thrown1.getClass === classOf[SparkException]) + assert(thrown1.getMessage.contains("NotSerializableException")) + + FailureSuiteState.clear() + } + // TODO: Need to add tests with shuffle fetch failures. } From 5ed38b4d1d7154235d5b72c35d3a8e63bac3a2de Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:59 -0700 Subject: [PATCH 119/221] Scheduler code style cleanup. --- .../cluster/ClusterTaskSetManager.scala | 154 +++++++++++------- .../scheduler/cluster/TaskSetManager.scala | 22 ++- .../scheduler/local/LocalScheduler.scala | 22 ++- .../scheduler/local/LocalTaskSetManager.scala | 82 +++++----- .../scheduler/ClusterSchedulerSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 13 +- 6 files changed, 170 insertions(+), 125 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 3d06520675..d934293b70 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -17,7 +17,8 @@ package spark.scheduler.cluster -import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} +import java.nio.ByteBuffer +import java.util.{Arrays, NoSuchElementException} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -25,12 +26,14 @@ import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import spark._ -import spark.scheduler._ +import spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils} +import spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} import spark.TaskState.TaskState -import java.nio.ByteBuffer +import spark.scheduler.{ShuffleMapTask, Task, TaskResult, TaskSet} -private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { + +private[spark] object TaskLocality + extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { // process local is expected to be used ONLY within tasksetmanager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value @@ -43,8 +46,10 @@ private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LO assert (constraint != TaskLocality.PROCESS_LOCAL) constraint match { - case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL - case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL + case TaskLocality.NODE_LOCAL => + condition == TaskLocality.NODE_LOCAL + case TaskLocality.RACK_LOCAL => + condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL // For anything else, allow case _ => true } @@ -56,11 +61,10 @@ private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LO val retval = TaskLocality.withName(str) // Must not specify PROCESS_LOCAL ! assert (retval != TaskLocality.PROCESS_LOCAL) - retval } catch { case nEx: NoSuchElementException => { - logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL"); + logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL") // default to preserve earlier behavior NODE_LOCAL } @@ -71,11 +75,8 @@ private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LO /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -private[spark] class ClusterTaskSetManager( - sched: ClusterScheduler, - val taskSet: TaskSet) - extends TaskSetManager - with Logging { +private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) + extends TaskSetManager with Logging { // Maximum time to wait to run a task in a preferred location (in ms) val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong @@ -106,13 +107,14 @@ private[spark] class ClusterTaskSetManager( var runningTasks = 0 var priority = taskSet.priority var stageId = taskSet.stageId - var name = "TaskSet_"+taskSet.stageId.toString - var parent:Schedulable = null + var name = "TaskSet_" + taskSet.stageId.toString + var parent: Schedulable = null // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis - // List of pending tasks for each node (process local to container). These collections are actually + // List of pending tasks for each node (process local to container). + // These collections are actually // treated as stacks, in which new tasks are added to the end of the // ArrayBuffer and removed from the end. This makes it faster to detect // tasks that repeatedly fail because whenever a task failed, it is put @@ -172,9 +174,11 @@ private[spark] class ClusterTaskSetManager( // Note that it follows the hierarchy. // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL - private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, - taskLocality: TaskLocality.TaskLocality): HashSet[String] = { - + private def findPreferredLocations( + _taskPreferredLocations: Seq[String], + scheduler: ClusterScheduler, + taskLocality: TaskLocality.TaskLocality): HashSet[String] = + { if (TaskLocality.PROCESS_LOCAL == taskLocality) { // straight forward comparison ! Special case it. val retval = new HashSet[String]() @@ -189,13 +193,14 @@ private[spark] class ClusterTaskSetManager( return retval } - val taskPreferredLocations = + val taskPreferredLocations = { if (TaskLocality.NODE_LOCAL == taskLocality) { _taskPreferredLocations } else { assert (TaskLocality.RACK_LOCAL == taskLocality) // Expand set to include all 'seen' rack local hosts. - // This works since container allocation/management happens within master - so any rack locality information is updated in msater. + // This works since container allocation/management happens within master - + // so any rack locality information is updated in msater. // Best case effort, and maybe sort of kludge for now ... rework it later ? val hosts = new HashSet[String] _taskPreferredLocations.foreach(h => { @@ -213,6 +218,7 @@ private[spark] class ClusterTaskSetManager( hosts } + } val retval = new HashSet[String] scheduler.synchronized { @@ -229,11 +235,13 @@ private[spark] class ClusterTaskSetManager( // Add a task to all the pending-task lists that it should be on. private def addPendingTask(index: Int) { - // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate - // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL) - val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + // We can infer hostLocalLocations from rackLocalLocations by joining it against + // tasks(index).preferredLocations (with appropriate hostPort <-> host conversion). + // But not doing it for simplicity sake. If this becomes a performance issue, modify it. + val locs = tasks(index).preferredLocations + val processLocalLocations = findPreferredLocations(locs, sched, TaskLocality.PROCESS_LOCAL) + val hostLocalLocations = findPreferredLocations(locs, sched, TaskLocality.NODE_LOCAL) + val rackLocalLocations = findPreferredLocations(locs, sched, TaskLocality.RACK_LOCAL) if (rackLocalLocations.size == 0) { // Current impl ensures this. @@ -298,18 +306,24 @@ private[spark] class ClusterTaskSetManager( } // Number of pending tasks for a given host Port (which would be process local) - def numPendingTasksForHostPort(hostPort: String): Int = { - getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + override def numPendingTasksForHostPort(hostPort: String): Int = { + getPendingTasksForHostPort(hostPort).count { index => + copiesRunning(index) == 0 && !finished(index) + } } // Number of pending tasks for a given host (which would be data local) - def numPendingTasksForHost(hostPort: String): Int = { - getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + override def numPendingTasksForHost(hostPort: String): Int = { + getPendingTasksForHost(hostPort).count { index => + copiesRunning(index) == 0 && !finished(index) + } } // Number of pending rack local tasks for a given host - def numRackLocalPendingTasksForHost(hostPort: String): Int = { - getRackLocalPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + override def numRackLocalPendingTasksForHost(hostPort: String): Int = { + getRackLocalPendingTasksForHost(hostPort).count { index => + copiesRunning(index) == 0 && !finished(index) + } } @@ -337,12 +351,12 @@ private[spark] class ClusterTaskSetManager( speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set if (speculatableTasks.size > 0) { - val localTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) - } + val localTask = speculatableTasks.find { index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, + TaskLocality.NODE_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) + } if (localTask != None) { speculatableTasks -= localTask.get @@ -351,11 +365,11 @@ private[spark] class ClusterTaskSetManager( // check for rack locality if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - val rackTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - locations.contains(hostPort) && !attemptLocs.contains(hostPort) + val rackTask = speculatableTasks.find { index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, + TaskLocality.RACK_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + locations.contains(hostPort) && !attemptLocs.contains(hostPort) } if (rackTask != None) { @@ -367,7 +381,9 @@ private[spark] class ClusterTaskSetManager( // Any task ... if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { // Check for attemptLocs also ? - val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.hostPort).contains(hostPort)) + val nonLocalTask = speculatableTasks.find { i => + !taskAttempts(i).map(_.hostPort).contains(hostPort) + } if (nonLocalTask != None) { speculatableTasks -= nonLocalTask.get return nonLocalTask @@ -397,7 +413,8 @@ private[spark] class ClusterTaskSetManager( } } - // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to failed tasks later rather than sooner. + // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to + // failed tasks later rather than sooner. // TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down). val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs) if (noPrefTask != None) { @@ -433,7 +450,8 @@ private[spark] class ClusterTaskSetManager( locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined } - // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). + // Does a host count as a rack local preferred location for a task? + // (assumes host is NOT preferred location). // This is true if either the task has preferred locations and this host is one, or it has // no preferred locations (in which we still count the launch as preferred). private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { @@ -454,14 +472,22 @@ private[spark] class ClusterTaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { - + override def slaveOffer( + execId: String, + hostPort: String, + availableCpus: Double, + overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = + { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { // If explicitly specified, use that val locality = if (overrideLocality != null) overrideLocality else { // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... val time = System.currentTimeMillis - if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY + if (time - lastPreferredLaunchTime < LOCALITY_WAIT) { + TaskLocality.NODE_LOCAL + } else { + TaskLocality.ANY + } } findTask(hostPort, locality) match { @@ -504,7 +530,7 @@ private[spark] class ClusterTaskSetManager( return None } - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { state match { case TaskState.FINISHED => taskFinished(tid, state, serializedData) @@ -536,7 +562,8 @@ private[spark] class ClusterTaskSetManager( try { val result = ser.deserialize[TaskResult[_]](serializedData) result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + sched.listener.taskEnded( + tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) } catch { case cnf: ClassNotFoundException => val loader = Thread.currentThread().getContextClassLoader @@ -582,8 +609,8 @@ private[spark] class ClusterTaskSetManager( return case taskResultTooBig: TaskResultTooBigFailure => - logInfo("Loss was due to task %s result exceeding Akka frame size; " + - "aborting job".format(tid)) + logInfo("Loss was due to task %s result exceeding Akka frame size; aborting job".format( + tid)) abort("Task %s result exceeded Akka frame size".format(tid)) return @@ -634,7 +661,7 @@ private[spark] class ClusterTaskSetManager( } } - def error(message: String) { + override def error(message: String) { // Save the error message abort("Error: " + message) } @@ -662,7 +689,8 @@ private[spark] class ClusterTaskSetManager( } } - //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + // TODO: for now we just find Pool not TaskSetManager, + // we can extend this function in future if needed override def getSchedulableByName(name: String): Schedulable = { return null } @@ -687,13 +715,15 @@ private[spark] class ClusterTaskSetManager( // If some task has preferred locations only on hostname, and there are no more executors there, // put it in the no-prefs list to avoid the wait from delay scheduling - // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to - // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. - // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if - // there is no host local node for the task (not if there is no process local node for the task) + // host local tasks - should we push this to rack local or no pref list ? For now, preserving + // behavior and moving to no prefs list. Note, this was done due to impliations related to + // 'waiting' for data local tasks, etc. + // Note: NOT checking process local list - since host local list is super set of that. We need + // to ad to no prefs only if there is no host local node for the task (not if there is no + // process local node for the task) for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { - // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) + val newLocs = findPreferredLocations( + tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) if (newLocs.isEmpty) { pendingTasksWithNoPrefs += index } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 07c3ddcc7e..7978a5df74 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -17,18 +17,28 @@ package spark.scheduler.cluster -import scala.collection.mutable.ArrayBuffer -import spark.scheduler._ -import spark.TaskState.TaskState import java.nio.ByteBuffer +import spark.TaskState.TaskState +import spark.scheduler.TaskSet + private[spark] trait TaskSetManager extends Schedulable { + def taskSet: TaskSet - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, - overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] + + def slaveOffer( + execId: String, + hostPort: String, + availableCpus: Double, + overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] + def numPendingTasksForHostPort(hostPort: String): Int - def numRackLocalPendingTasksForHost(hostPort :String): Int + + def numRackLocalPendingTasksForHost(hostPort: String): Int + def numPendingTasksForHost(hostPort: String): Int + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) + def error(message: String) } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 1f73cb99a7..edd83d4cb4 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -37,10 +37,15 @@ import akka.actor._ * testing fault recovery. */ -private[spark] case class LocalReviveOffers() -private[spark] case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) +private[spark] +case class LocalReviveOffers() + +private[spark] +case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) + +private[spark] +class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { -private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { def receive = { case LocalReviveOffers => launchTask(localScheduler.resourceOffer(freeCores)) @@ -55,7 +60,7 @@ private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: I freeCores -= 1 localScheduler.threadPool.submit(new Runnable { def run() { - localScheduler.runTask(task.taskId,task.serializedTask) + localScheduler.runTask(task.taskId, task.serializedTask) } }) } @@ -110,7 +115,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: override def submitTasks(taskSet: TaskSet) { synchronized { - var manager = new LocalTaskSetManager(this, taskSet) + val manager = new LocalTaskSetManager(this, taskSet) schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) activeTaskSets(taskSet.id) = manager taskSetTaskIds(taskSet.id) = new HashSet[Long]() @@ -124,14 +129,15 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val tasks = new ArrayBuffer[TaskDescription](freeCores) val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() for (manager <- sortedTaskSetQueue) { - logDebug("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) + logDebug("parentName:%s,name:%s,runningTasks:%s".format( + manager.parent.name, manager.name, manager.runningTasks)) } var launchTask = false for (manager <- sortedTaskSetQueue) { do { launchTask = false - manager.slaveOffer(null,null,freeCpuCores) match { + manager.slaveOffer(null, null, freeCpuCores) match { case Some(task) => tasks += task taskIdToTaskSetId(task.taskId) = manager.taskSet.id @@ -139,7 +145,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: freeCpuCores -= 1 launchTask = true case None => {} - } + } } while(launchTask) } return tasks diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index e662ad6709..bbce9eda64 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -17,27 +17,26 @@ package spark.scheduler.local -import java.io.File -import java.util.concurrent.atomic.AtomicInteger import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import spark._ +import spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState} import spark.TaskState.TaskState -import spark.scheduler._ -import spark.scheduler.cluster._ +import spark.scheduler.{Task, TaskResult, TaskSet} +import spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager} + + +private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) + extends TaskSetManager with Logging { -private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { var parent: Schedulable = null var weight: Int = 1 var minShare: Int = 0 var runningTasks: Int = 0 var priority: Int = taskSet.priority var stageId: Int = taskSet.stageId - var name: String = "TaskSet_"+taskSet.stageId.toString - + var name: String = "TaskSet_" + taskSet.stageId.toString var failCount = new Array[Int](taskSet.tasks.size) val taskInfos = new HashMap[Long, TaskInfo] @@ -49,49 +48,45 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val numFailures = new Array[Int](numTasks) val MAX_TASK_FAILURES = sched.maxFailures - def increaseRunningTasks(taskNum: Int): Unit = { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } + override def increaseRunningTasks(taskNum: Int): Unit = { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } } - def decreaseRunningTasks(taskNum: Int): Unit = { + override def decreaseRunningTasks(taskNum: Int): Unit = { runningTasks -= taskNum if (parent != null) { parent.decreaseRunningTasks(taskNum) } } - def addSchedulable(schedulable: Schedulable): Unit = { + override def addSchedulable(schedulable: Schedulable): Unit = { //nothing } - def removeSchedulable(schedulable: Schedulable): Unit = { + override def removeSchedulable(schedulable: Schedulable): Unit = { //nothing } - def getSchedulableByName(name: String): Schedulable = { + override def getSchedulableByName(name: String): Schedulable = { return null } - def executorLost(executorId: String, host: String): Unit = { + override def executorLost(executorId: String, host: String): Unit = { //nothing } - def checkSpeculatableTasks(): Boolean = { - return true - } + override def checkSpeculatableTasks() = true - def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] sortedTaskSetQueue += this return sortedTaskSetQueue } - def hasPendingTasks(): Boolean = { - return true - } + override def hasPendingTasks() = true def findTask(): Option[Int] = { for (i <- 0 to numTasks-1) { @@ -102,17 +97,25 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + override def slaveOffer( + execId: String, + hostPort: String, + availableCpus: Double, + overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = + { SparkEnv.set(sched.env) - logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format(availableCpus.toInt, numFinished, numTasks)) + logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format( + availableCpus.toInt, numFinished, numTasks)) if (availableCpus > 0 && numFinished < numTasks) { findTask() match { case Some(index) => val taskId = sched.attemptId.getAndIncrement() val task = taskSet.tasks(index) - val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", + TaskLocality.NODE_LOCAL) taskInfos(taskId) = info - val bytes = Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + val bytes = Task.serializeWithDependencies( + task, sched.sc.addedFiles, sched.sc.addedJars, ser) logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") val taskName = "task %s:%d".format(taskSet.id, index) copiesRunning(index) += 1 @@ -124,19 +127,19 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - def numPendingTasksForHostPort(hostPort: String): Int = { + override def numPendingTasksForHostPort(hostPort: String): Int = { return 0 } - def numRackLocalPendingTasksForHost(hostPort :String): Int = { + override def numRackLocalPendingTasksForHost(hostPort :String): Int = { return 0 } - def numPendingTasksForHost(hostPort: String): Int = { + override def numPendingTasksForHost(hostPort: String): Int = { return 0 } - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { state match { case TaskState.FINISHED => taskEnded(tid, state, serializedData) @@ -168,15 +171,18 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val task = taskSet.tasks(index) info.markFailed() decreaseRunningTasks(1) - val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) + val reason: ExceptionFailure = ser.deserialize[ExceptionFailure]( + serializedData, getClass.getClassLoader) sched.listener.taskEnded(task, reason, null, null, info, reason.metrics.getOrElse(null)) if (!finished(index)) { copiesRunning(index) -= 1 numFailures(index) += 1 val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s\n%s".format(reason.className, reason.description, locs.mkString("\n"))) + logInfo("Loss was due to %s\n%s\n%s".format( + reason.className, reason.description, locs.mkString("\n"))) if (numFailures(index) > MAX_TASK_FAILURES) { - val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(taskSet.id, index, 4, reason.description) + val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format( + taskSet.id, index, 4, reason.description) decreaseRunningTasks(runningTasks) sched.listener.taskSetFailed(taskSet, errorMessage) // need to delete failed Taskset from schedule queue @@ -185,6 +191,6 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } } - def error(message: String) { + override def error(message: String) { } } diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 8f81d0b6ee..05afcd6567 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -33,7 +33,7 @@ class DummyTaskSetManager( initNumTasks: Int, clusterScheduler: ClusterScheduler, taskSet: TaskSet) - extends ClusterTaskSetManager(clusterScheduler,taskSet) { + extends ClusterTaskSetManager(clusterScheduler, taskSet) { parent = null weight = 1 diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index f802b66cf1..a8b88d7936 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -23,21 +23,14 @@ import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import spark.LocalSparkContext - -import spark.storage.BlockManager -import spark.storage.BlockManagerId -import spark.storage.BlockManagerMaster -import spark.{Dependency, ShuffleDependency, OneToOneDependency} -import spark.FetchFailedException import spark.MapOutputTracker import spark.RDD import spark.SparkContext -import spark.SparkException import spark.Partition import spark.TaskContext -import spark.TaskEndReason - -import spark.{FetchFailed, Success} +import spark.{Dependency, ShuffleDependency, OneToOneDependency} +import spark.{FetchFailed, Success, TaskEndReason} +import spark.storage.{BlockManagerId, BlockManagerMaster} /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler From f2422d4f29abb80b9bc76c4596d1cc31d9e6d590 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:20 -0700 Subject: [PATCH 120/221] SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure. --- .../scala/spark/scheduler/DAGScheduler.scala | 29 +++++++++++++------ core/src/test/scala/spark/FailureSuite.scala | 29 ++++++++++++++----- 2 files changed, 42 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 29e879aa42..5fcd807aff 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -17,19 +17,17 @@ package spark.scheduler -import cluster.TaskInfo -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.LinkedBlockingQueue -import java.util.concurrent.TimeUnit +import java.io.NotSerializableException import java.util.Properties +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ import spark.executor.TaskMetrics -import spark.partial.ApproximateActionListener -import spark.partial.ApproximateEvaluator -import spark.partial.PartialResult +import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} +import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -258,7 +256,8 @@ class DAGScheduler( assert(partitions.size > 0) val waiter = new JobWaiter(partitions.size, resultHandler) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) + val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, + properties) return (toSubmit, waiter) } @@ -283,7 +282,7 @@ class DAGScheduler( "Total number of partitions: " + maxPartitions) } - val (toSubmit, waiter) = prepareJob( + val (toSubmit: JobSubmitted, waiter: JobWaiter[_]) = prepareJob( finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) waiter.awaitResult() match { @@ -466,6 +465,18 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") + + // Preemptively serialize the stage RDD to make sure the tasks for this stage will be + // serializable. We are catching this exception here because it would be fairly hard to + // catch the non-serializable exception down the road, where we have several different + // implementations for local scheduler and cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) + } catch { + case e: NotSerializableException => abortStage(stage, e.toString) + return + } + if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index 6c847b8fef..c3c52f9118 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -18,9 +18,6 @@ package spark import org.scalatest.FunSuite -import org.scalatest.prop.Checkers - -import scala.collection.mutable.ArrayBuffer import SparkContext._ @@ -40,7 +37,7 @@ object FailureSuiteState { } class FailureSuite extends FunSuite with LocalSparkContext { - + // Run a 3-task map job in which task 1 deterministically fails once, and check // whether the job completes successfully and we ran 4 tasks in total. test("failure in a single-stage job") { @@ -66,7 +63,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure in a two-stage job") { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map { - case (k, v) => + case (k, v) => FailureSuiteState.synchronized { FailureSuiteState.tasksRun += 1 if (k == 1 && FailureSuiteState.tasksFailed == 0) { @@ -87,15 +84,33 @@ class FailureSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => new NonSerializable) - val thrown = intercept[spark.SparkException] { + val thrown = intercept[SparkException] { results.collect() } - assert(thrown.getClass === classOf[spark.SparkException]) + assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) FailureSuiteState.clear() } + test("failure because task closure is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + val thrown = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => a).count() + } + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("NotSerializableException")) + + val thrown1 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() + } + assert(thrown1.getClass === classOf[SparkException]) + assert(thrown1.getMessage.contains("NotSerializableException")) + + FailureSuiteState.clear() + } + // TODO: Need to add tests with shuffle fetch failures. } From 383684daaa62bebc177b4c74573ce509f154f74b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:33:27 -0700 Subject: [PATCH 121/221] Replaces Seq with HashSet, removes redundant import --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index bb2b003486..606e1eb2fc 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -5,7 +5,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Properties import spark.{ExceptionFailure, Logging, SparkContext, Success, Utils} @@ -18,7 +18,6 @@ import spark.ui.JettyUtils._ import spark.ui.Page.Executors import spark.ui.UIUtils.headerSparkPage import spark.ui.UIUtils -import spark.Utils import scala.xml.{Node, XML} @@ -114,7 +113,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, Seq[Long]]() + val executorToTasksActive = HashMap[String, HashSet[Long]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = @@ -122,12 +121,14 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()) :+ taskStart.taskInfo.taskId + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) + + taskStart.taskInfo.taskId } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()).filterNot(_ == taskEnd.taskInfo.taskId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) - + taskEnd.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => From abc78cd3318fb7bc69d10fd5422d20b299a8d7d8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:47:16 -0700 Subject: [PATCH 122/221] Modifies instead of copies HashSets, fixes comment style --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index b02bf8f4bf..7bf50de660 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -52,7 +52,7 @@ class DAGScheduler( } taskSched.setListener(this) - //Called by TaskScheduler to report task's starting. + // Called by TaskScheduler to report task's starting. override def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventQueue.put(BeginEvent(task, taskInfo)) } diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 606e1eb2fc..69fb306074 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -121,14 +121,16 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) + - taskStart.taskInfo.taskId + if (!executorToTasksActive.contains(eid)) + executorToTasksActive(eid) = HashSet[Long]() + executorToTasksActive(eid) += taskStart.taskInfo.taskId } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) - - taskEnd.taskInfo.taskId + if (!executorToTasksActive.contains(eid)) + executorToTasksActive(eid) = HashSet[Long]() + executorToTasksActive(eid) -= taskStart.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => From 6a31b7191d5a8203563a2a4e600210c67439abf5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 23 Jul 2013 16:20:24 -0700 Subject: [PATCH 123/221] Small bug fix --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 69fb306074..bad5c442ab 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -130,7 +130,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val eid = taskEnd.taskInfo.executorId if (!executorToTasksActive.contains(eid)) executorToTasksActive(eid) = HashSet[Long]() - executorToTasksActive(eid) -= taskStart.taskInfo.taskId + executorToTasksActive(eid) -= taskEnd.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -146,4 +146,4 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { executorToTaskInfos(eid) = taskList } } -} \ No newline at end of file +} From 85ab8114bc1367a0f4f32d4b8635c41aa547bc72 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:25:58 -0700 Subject: [PATCH 124/221] Moved non-serializable closure catching exception from submitStage to submitMissingTasks --- .../scala/spark/scheduler/DAGScheduler.scala | 25 ++++++++++--------- core/src/test/scala/spark/FailureSuite.scala | 10 ++++++++ 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 5fcd807aff..fde998494f 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -465,18 +465,6 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") - - // Preemptively serialize the stage RDD to make sure the tasks for this stage will be - // serializable. We are catching this exception here because it would be fairly hard to - // catch the non-serializable exception down the road, where we have several different - // implementations for local scheduler and cluster schedulers. - try { - SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) - } catch { - case e: NotSerializableException => abortStage(stage, e.toString) - return - } - if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) @@ -515,6 +503,19 @@ class DAGScheduler( } } if (tasks.size > 0) { + // Preemptively serialize a task to make sure it can be serialized. We are catching this + // exception here because it would be fairly hard to catch the non-serializable exception + // down the road, where we have several different implementations for local scheduler and + // cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) + } catch { + case e: NotSerializableException => + abortStage(stage, e.toString) + running -= stage + return + } + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index c3c52f9118..5b133cdd6e 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -96,18 +96,28 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure because task closure is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable + + // Non-serializable closure in the final result stage val thrown = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => a).count() } assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) + // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() } assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) + // Non-serializable closure in foreach function + val thrown2 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).foreach(x => println(a)) + } + assert(thrown2.getClass === classOf[SparkException]) + assert(thrown2.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() } From d33b8a2a0ffd6d085cbd8de22863a1f35c106270 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:28:39 -0700 Subject: [PATCH 125/221] Added comments on task closure serialization. --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 ++ .../main/scala/spark/scheduler/local/LocalTaskSetManager.scala | 2 ++ 2 files changed, 4 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index d934293b70..f64818876b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -515,6 +515,8 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: } // Serialize and return the task val startTime = System.currentTimeMillis + // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here + // we assume the task can be serialized without exceptions. val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) val timeTaken = System.currentTimeMillis - startTime diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index bbce9eda64..a9b49cad0e 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -114,6 +114,8 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) taskInfos(taskId) = info + // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here + // we assume the task can be serialized without exceptions. val bytes = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") From 3dae1df66f34d3716c697f85cfe4aedeee428688 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:25:58 -0700 Subject: [PATCH 126/221] Moved non-serializable closure catching exception from submitStage to submitMissingTasks --- .../scala/spark/scheduler/DAGScheduler.scala | 25 ++++++++++--------- core/src/test/scala/spark/FailureSuite.scala | 10 ++++++++ 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 5fcd807aff..fde998494f 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -465,18 +465,6 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") - - // Preemptively serialize the stage RDD to make sure the tasks for this stage will be - // serializable. We are catching this exception here because it would be fairly hard to - // catch the non-serializable exception down the road, where we have several different - // implementations for local scheduler and cluster schedulers. - try { - SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) - } catch { - case e: NotSerializableException => abortStage(stage, e.toString) - return - } - if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) @@ -515,6 +503,19 @@ class DAGScheduler( } } if (tasks.size > 0) { + // Preemptively serialize a task to make sure it can be serialized. We are catching this + // exception here because it would be fairly hard to catch the non-serializable exception + // down the road, where we have several different implementations for local scheduler and + // cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) + } catch { + case e: NotSerializableException => + abortStage(stage, e.toString) + running -= stage + return + } + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index c3c52f9118..5b133cdd6e 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -96,18 +96,28 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure because task closure is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable + + // Non-serializable closure in the final result stage val thrown = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => a).count() } assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) + // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() } assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) + // Non-serializable closure in foreach function + val thrown2 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).foreach(x => println(a)) + } + assert(thrown2.getClass === classOf[SparkException]) + assert(thrown2.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() } From 503acd3a379a3686d343fdf072fc231b8fba78f9 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:47:27 +0800 Subject: [PATCH 127/221] Build metrics system framwork --- conf/metrics.properties | 13 ++++ .../metrics/AbstractInstrumentation.scala | 68 +++++++++++++++++++ .../scala/spark/metrics/MetricsConfig.scala | 55 +++++++++++++++ .../spark/metrics/sink/ConsoleSink.scala | 43 ++++++++++++ .../scala/spark/metrics/sink/CsvSink.scala | 53 +++++++++++++++ .../scala/spark/metrics/sink/JmxSink.scala | 17 +++++ .../main/scala/spark/metrics/sink/Sink.scala | 6 ++ 7 files changed, 255 insertions(+) create mode 100644 conf/metrics.properties create mode 100644 core/src/main/scala/spark/metrics/AbstractInstrumentation.scala create mode 100644 core/src/main/scala/spark/metrics/MetricsConfig.scala create mode 100644 core/src/main/scala/spark/metrics/sink/ConsoleSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/CsvSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/JmxSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/Sink.scala diff --git a/conf/metrics.properties b/conf/metrics.properties new file mode 100644 index 0000000000..78749cf381 --- /dev/null +++ b/conf/metrics.properties @@ -0,0 +1,13 @@ +# syntax: [prefix].[sink].[instance].[options] + +*.sink.console.period=10 + +*.sink.console.unit=second + +master.sink.console.period=10 + +master.sink.console.unit=second + +worker.sink.console.period=20 + +worker.sink.console.unit=second diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala new file mode 100644 index 0000000000..0fed608488 --- /dev/null +++ b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala @@ -0,0 +1,68 @@ +package spark.metrics + +import scala.collection.mutable + +import com.codahale.metrics.{JmxReporter, MetricRegistry} + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import spark.Logging +import spark.metrics.sink._ + +trait AbstractInstrumentation extends Logging { + initLogging() + + def registryHandler: MetricRegistry + def instance: String + + val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) + val metricsConfig = new MetricsConfig(confFile) + + val sinks = new mutable.ArrayBuffer[Sink] + + def registerSinks() { + val instConfig = metricsConfig.getInstance(instance) + val sinkConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SINK_REGEX) + + // Register JMX sink as a default sink + sinks += new JmxSink(registryHandler) + + // Register other sinks according to conf + sinkConfigs.foreach { kv => + val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { + AbstractInstrumentation.DEFAULT_SINKS(kv._1) + } else { + kv._2.getProperty("class") + } + try { + val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) + .newInstance(kv._2, registryHandler) + sinks += sink.asInstanceOf[Sink] + } catch { + case e: Exception => logError("class " + classPath + "cannot be instantialize", e) + } + } + + sinks.foreach(_.registerSink) + } + + def unregisterSinks() { + sinks.foreach(_.unregisterSink) + } +} + +object AbstractInstrumentation { + val DEFAULT_SINKS = Map( + "console" -> "spark.metrics.sink.ConsoleSink", + "csv" -> "spark.metrics.sink.CsvSink") + + val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + + val timeUnits = Map( + "millisecond" -> TimeUnit.MILLISECONDS, + "second" -> TimeUnit.SECONDS, + "minute" -> TimeUnit.MINUTES, + "hour" -> TimeUnit.HOURS, + "day" -> TimeUnit.DAYS) +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala new file mode 100644 index 0000000000..0fec1988ea --- /dev/null +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -0,0 +1,55 @@ +package spark.metrics + +import java.util.Properties +import java.io.FileInputStream + +import scala.collection.mutable +import scala.util.matching.Regex + +class MetricsConfig(val configFile: String) { + val properties = new Properties() + var fis: FileInputStream = _ + + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } + + val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) + if (propertyCategories.contains(MetricsConfig.DEFAULT_PREFIX)) { + import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(MetricsConfig.DEFAULT_PREFIX) + for ((inst, prop) <- propertyCategories; p <- defaultProperty + if inst != MetricsConfig.DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { + prop.setProperty(p._1, p._2) + } + } + + def getInstance(inst: String) = { + propertyCategories.get(inst) match { + case Some(s) => s + case None => propertyCategories(MetricsConfig.DEFAULT_PREFIX) + } + } +} + +object MetricsConfig { + val DEFAULT_CONFIG_FILE = "/home/jerryshao/project/sotc_cloud-spark/conf/metrics.properties" + val DEFAULT_PREFIX = "*" + val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + + def subProperties(prop: Properties, regex: Regex) = { + val subProperties = new mutable.HashMap[String, Properties] + + import scala.collection.JavaConversions._ + prop.foreach { kv => + val regex(a, b) = kv._1 + subProperties.getOrElseUpdate(a, new Properties).setProperty(b, kv._2) + println(">>>>>subProperties added " + a + " " + b + " " + kv._2) + } + + subProperties + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala new file mode 100644 index 0000000000..5426af8c4c --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -0,0 +1,43 @@ +package spark.metrics.sink + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => ConsoleSink.CONSOLE_DEFAULT_PERIOD.toInt + } + + val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { + case Some(s) => AbstractInstrumentation.timeUnits(s) + case None => AbstractInstrumentation.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + } + + var reporter: ConsoleReporter = _ + + override def registerSink() { + reporter = ConsoleReporter.forRegistry(registry) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build() + + reporter.start(pollPeriod, pollUnit) + } + + override def unregisterSink() { + reporter.stop() + } +} + +object ConsoleSink { + val CONSOLE_DEFAULT_PERIOD = "10" + val CONSOLE_DEFAULT_UNIT = "second" + + val CONSOLE_KEY_PERIOD = "period" + val CONSOLE_KEY_UNIT = "unit" +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala new file mode 100644 index 0000000000..3a80c36901 --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -0,0 +1,53 @@ +package spark.metrics.sink + +import java.io.File +import java.util.{Locale, Properties} +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.{CsvReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => CsvSink.CSV_DEFAULT_PERIOD.toInt + } + + val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { + case Some(s) => AbstractInstrumentation.timeUnits(s) + case None => AbstractInstrumentation.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + } + + val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { + case Some(s) => s + case None => CsvSink.CSV_DEFAULT_DIR + } + + var reporter: CsvReporter = _ + + override def registerSink() { + reporter = CsvReporter.forRegistry(registry) + .formatFor(Locale.US) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build(new File(pollDir)) + + reporter.start(pollPeriod, pollUnit) + } + + override def unregisterSink() { + reporter.stop() + } +} + +object CsvSink { + val CSV_KEY_PERIOD = "period" + val CSV_KEY_UNIT = "unit" + val CSV_KEY_DIR = "directory" + + val CSV_DEFAULT_PERIOD = "1" + val CSV_DEFAULT_UNIT = "minute" + val CSV_DEFAULT_DIR = "/tmp/" +} + diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala new file mode 100644 index 0000000000..56e5677700 --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -0,0 +1,17 @@ +package spark.metrics.sink + +import com.codahale.metrics.{JmxReporter, MetricRegistry} + +class JmxSink(registry: MetricRegistry) extends Sink { + var reporter: JmxReporter = _ + + override def registerSink() { + reporter = JmxReporter.forRegistry(registry).build() + reporter.start() + } + + override def unregisterSink() { + reporter.stop() + } + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala new file mode 100644 index 0000000000..65ebcb4eac --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -0,0 +1,6 @@ +package spark.metrics.sink + +trait Sink { + def registerSink: Unit + def unregisterSink: Unit +} \ No newline at end of file From 9dec8c73e6f0c3b6b55a11ff92cc9bff18dadd24 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:48:41 +0800 Subject: [PATCH 128/221] Add Master and Worker instrumentation support --- .../scala/spark/deploy/master/Master.scala | 9 ++- .../deploy/master/MasterInstrumentation.scala | 44 +++++++++++++++ .../scala/spark/deploy/worker/Worker.scala | 11 +++- .../deploy/worker/WorkerInstrumentation.scala | 55 +++++++++++++++++++ 4 files changed, 115 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala create mode 100644 core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index eddcafd84d..3a7c4e5a52 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -33,7 +33,8 @@ import spark.util.AkkaUtils import ui.MasterWebUI -private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { +private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor +with Logging with MasterInstrumentation { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 @@ -73,6 +74,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) + + initialize(this) } override def postStop() { @@ -316,6 +319,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } + + override def postStop() { + uninitialize() + } } private[spark] object Master { diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala new file mode 100644 index 0000000000..13088189a4 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -0,0 +1,44 @@ +package spark.deploy.master + +import com.codahale.metrics.{Gauge, JmxReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +private[spark] trait MasterInstrumentation extends AbstractInstrumentation { + var masterInst: Option[Master] = None + val metricRegistry = new MetricRegistry() + + override def registryHandler = metricRegistry + + override def instance = "master" + + def initialize(master: Master) { + masterInst = Some(master) + + // Register and start all the sinks + registerSinks + } + + def uninitialize() { + unregisterSinks + } + + // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "workers", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.workers.size).getOrElse(0) + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "apps", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.apps.size).getOrElse(0) + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "waiting_apps", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.waitingApps.size).getOrElse(0) + }) + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 0bd88ea253..b64bdb8d28 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -41,7 +41,7 @@ private[spark] class Worker( memory: Int, masterUrl: String, workDirPath: String = null) - extends Actor with Logging { + extends Actor with Logging with WorkerInstrumentation { Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -97,6 +97,9 @@ private[spark] class Worker( webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() + startWebUi() + + initialize(this) } def connectToMaster() { @@ -155,10 +158,10 @@ private[spark] class Worker( case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => masterDisconnected() - + case RequestWorkerState => { sender ! WorkerState(host, port, workerId, executors.values.toList, - finishedExecutors.values.toList, masterUrl, cores, memory, + finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed, masterWebUiUrl) } } @@ -178,6 +181,8 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) webUi.stop() + + uninitialize() } } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala new file mode 100644 index 0000000000..04c43ce33b --- /dev/null +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -0,0 +1,55 @@ +package spark.deploy.worker + +import com.codahale.metrics.{JmxReporter, Gauge, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { + var workerInst: Option[Worker] = None + val metricRegistry = new MetricRegistry() + + override def registryHandler = metricRegistry + + override def instance = "worker" + + def initialize(worker: Worker) { + workerInst = Some(worker) + + // Register and start all the sinks + registerSinks() + } + + def uninitialize() { + unregisterSinks() + } + + // Gauge for executors number + metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) + }) + + // Gauge for cores used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "Mbytes"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) + }) + + // Gauge for cores free of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) + }) +} \ No newline at end of file From c3daad3f65630eb4ed536d06c0d467cde57a8142 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 12:00:19 +0800 Subject: [PATCH 129/221] Update metric source support for instrumentation --- conf/metrics.properties | 4 ++- .../deploy/master/MasterInstrumentation.scala | 7 +++-- .../deploy/worker/WorkerInstrumentation.scala | 5 +++- .../metrics/AbstractInstrumentation.scala | 29 +++++++++++++++++-- .../scala/spark/metrics/MetricsConfig.scala | 12 ++++---- .../scala/spark/metrics/sink/CsvSink.scala | 4 +-- .../main/scala/spark/metrics/sink/Sink.scala | 1 + .../spark/metrics/source/JvmSource.scala | 17 +++++++++++ .../scala/spark/metrics/source/Source.scala | 5 ++++ 9 files changed, 70 insertions(+), 14 deletions(-) create mode 100644 core/src/main/scala/spark/metrics/source/JvmSource.scala create mode 100644 core/src/main/scala/spark/metrics/source/Source.scala diff --git a/conf/metrics.properties b/conf/metrics.properties index 78749cf381..0bbb6b5229 100644 --- a/conf/metrics.properties +++ b/conf/metrics.properties @@ -1,9 +1,11 @@ -# syntax: [prefix].[sink].[instance].[options] +# syntax: [prefix].[sink|source].[instance].[options] *.sink.console.period=10 *.sink.console.unit=second +master.source.jvm.class=spark.metrics.source.JvmSource + master.sink.console.period=10 master.sink.console.unit=second diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 13088189a4..c295e725d7 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -15,12 +15,15 @@ private[spark] trait MasterInstrumentation extends AbstractInstrumentation { def initialize(master: Master) { masterInst = Some(master) + // Register all the sources + registerSources() + // Register and start all the sinks - registerSinks + registerSinks() } def uninitialize() { - unregisterSinks + unregisterSinks() } // Gauge for worker numbers in cluster diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 04c43ce33b..2f725300b5 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -15,6 +15,9 @@ private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { def initialize(worker: Worker) { workerInst = Some(worker) + // Register all the sources + registerSources() + // Register and start all the sinks registerSinks() } @@ -36,7 +39,7 @@ private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "Mbytes"), + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), new Gauge[Int] { override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) }) diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala index 0fed608488..9cae1e0220 100644 --- a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala +++ b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala @@ -9,17 +9,39 @@ import java.util.concurrent.TimeUnit import spark.Logging import spark.metrics.sink._ +import spark.metrics.source._ -trait AbstractInstrumentation extends Logging { +private [spark] trait AbstractInstrumentation extends Logging { initLogging() + // Get MetricRegistry handler def registryHandler: MetricRegistry + // Get the instance name def instance: String val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] + val sources = new mutable.ArrayBuffer[Source] + + def registerSources() { + val instConfig = metricsConfig.getInstance(instance) + val sourceConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SOURCE_REGEX) + + // Register all the sources + sourceConfigs.foreach { kv => + val classPath = kv._2.getProperty("class") + try { + val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) + .newInstance(registryHandler) + sources += source.asInstanceOf[Source] + } catch { + case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) + } + } + sources.foreach(_.registerSource) + } def registerSinks() { val instConfig = metricsConfig.getInstance(instance) @@ -33,6 +55,7 @@ trait AbstractInstrumentation extends Logging { val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { AbstractInstrumentation.DEFAULT_SINKS(kv._1) } else { + // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") } try { @@ -40,10 +63,9 @@ trait AbstractInstrumentation extends Logging { .newInstance(kv._2, registryHandler) sinks += sink.asInstanceOf[Sink] } catch { - case e: Exception => logError("class " + classPath + "cannot be instantialize", e) + case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.registerSink) } @@ -58,6 +80,7 @@ object AbstractInstrumentation { "csv" -> "spark.metrics.sink.CsvSink") val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r val timeUnits = Map( "millisecond" -> TimeUnit.MILLISECONDS, diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 0fec1988ea..be4f670918 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -6,7 +6,7 @@ import java.io.FileInputStream import scala.collection.mutable import scala.util.matching.Regex -class MetricsConfig(val configFile: String) { +private [spark] class MetricsConfig(val configFile: String) { val properties = new Properties() var fis: FileInputStream = _ @@ -36,7 +36,7 @@ class MetricsConfig(val configFile: String) { } object MetricsConfig { - val DEFAULT_CONFIG_FILE = "/home/jerryshao/project/sotc_cloud-spark/conf/metrics.properties" + val DEFAULT_CONFIG_FILE = "conf/metrics.properties" val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r @@ -45,9 +45,11 @@ object MetricsConfig { import scala.collection.JavaConversions._ prop.foreach { kv => - val regex(a, b) = kv._1 - subProperties.getOrElseUpdate(a, new Properties).setProperty(b, kv._2) - println(">>>>>subProperties added " + a + " " + b + " " + kv._2) + if (regex.findPrefixOf(kv._1) != None) { + val regex(prefix, suffix) = kv._1 + subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) + println(">>>>>subProperties added " + prefix + " " + suffix + " " + kv._2) + } } subProperties diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 3a80c36901..1d663f6cff 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -46,8 +46,8 @@ object CsvSink { val CSV_KEY_UNIT = "unit" val CSV_KEY_DIR = "directory" - val CSV_DEFAULT_PERIOD = "1" - val CSV_DEFAULT_UNIT = "minute" + val CSV_DEFAULT_PERIOD = "10" + val CSV_DEFAULT_UNIT = "second" val CSV_DEFAULT_DIR = "/tmp/" } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 65ebcb4eac..26052b7231 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -2,5 +2,6 @@ package spark.metrics.sink trait Sink { def registerSink: Unit + def unregisterSink: Unit } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala new file mode 100644 index 0000000000..8f6bf48843 --- /dev/null +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -0,0 +1,17 @@ +package spark.metrics.source + +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} + +class JvmSource(registry: MetricRegistry) extends Source { + // Initialize memory usage gauge for jvm + val memUsageMetricSet = new MemoryUsageGaugeSet + + // Initialize garbage collection usage gauge for jvm + val gcMetricSet = new GarbageCollectorMetricSet + + override def registerSource() { + registry.registerAll(memUsageMetricSet) + registry.registerAll(gcMetricSet) + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala new file mode 100644 index 0000000000..35cfe0c8ff --- /dev/null +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -0,0 +1,5 @@ +package spark.metrics.source + +trait Source { + def registerSource: Unit +} \ No newline at end of file From 03f9871116801abbdd7b4c7892c8d6affb1c4d9e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 18:29:54 +0800 Subject: [PATCH 130/221] MetricsSystem refactor --- .../scala/spark/deploy/master/Master.scala | 17 ++- .../deploy/master/MasterInstrumentation.scala | 66 ++++----- .../scala/spark/deploy/worker/Worker.scala | 12 +- .../deploy/worker/WorkerInstrumentation.scala | 131 +++++++++++------- ...trumentation.scala => MetricsSystem.scala} | 54 +++++--- .../spark/metrics/sink/ConsoleSink.scala | 10 +- .../scala/spark/metrics/sink/CsvSink.scala | 10 +- .../scala/spark/metrics/sink/JmxSink.scala | 4 +- .../main/scala/spark/metrics/sink/Sink.scala | 4 +- .../spark/metrics/source/JvmSource.scala | 30 ++-- .../scala/spark/metrics/source/Source.scala | 6 +- 11 files changed, 194 insertions(+), 150 deletions(-) rename core/src/main/scala/spark/metrics/{AbstractInstrumentation.scala => MetricsSystem.scala} (64%) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3a7c4e5a52..e44f5e3168 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -29,12 +29,12 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import spark.deploy._ import spark.{Logging, SparkException, Utils} +import spark.metrics.MetricsSystem import spark.util.AkkaUtils import ui.MasterWebUI -private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor -with Logging with MasterInstrumentation { +private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 @@ -57,6 +57,8 @@ with Logging with MasterInstrumentation { val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") + + val masterInstrumentation = new MasterInstrumentation(this) val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -75,7 +77,7 @@ with Logging with MasterInstrumentation { webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - initialize(this) + Master.metricsSystem.registerSource(masterInstrumentation) } override def postStop() { @@ -319,21 +321,22 @@ with Logging with MasterInstrumentation { removeWorker(worker) } } - - override def postStop() { - uninitialize() - } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r + + private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) + + metricsSystem.start() actorSystem.awaitTermination() + metricsSystem.stop() } /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index c295e725d7..5ea9a90319 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -1,47 +1,35 @@ package spark.deploy.master -import com.codahale.metrics.{Gauge, JmxReporter, MetricRegistry} +import java.util.{Map, HashMap => JHashMap} -import spark.metrics.AbstractInstrumentation +import com.codahale.metrics.{Gauge, Metric} -private[spark] trait MasterInstrumentation extends AbstractInstrumentation { - var masterInst: Option[Master] = None - val metricRegistry = new MetricRegistry() - - override def registryHandler = metricRegistry - - override def instance = "master" - - def initialize(master: Master) { - masterInst = Some(master) +import spark.metrics.source.Source + +private[spark] class MasterInstrumentation(val master: Master) extends Source { + val className = classOf[Master].getName() + val instrumentationName = "master" - // Register all the sources - registerSources() + override def sourceName = instrumentationName + + override def getMetrics(): Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] - // Register and start all the sinks - registerSinks() + // Gauge for worker numbers in cluster + gauges.put(className + ".workers.number", new Gauge[Int] { + override def getValue: Int = master.workers.size + }) + + // Gauge for application numbers in cluster + gauges.put(className + ".apps.number", new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + gauges.put(className + ".waiting_apps.number", new Gauge[Int] { + override def getValue: Int = master.waitingApps.size + }) + + gauges } - - def uninitialize() { - unregisterSinks() - } - - // Gauge for worker numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "workers", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.workers.size).getOrElse(0) - }) - - // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "apps", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.apps.size).getOrElse(0) - }) - - // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "waiting_apps", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.waitingApps.size).getOrElse(0) - }) - } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index b64bdb8d28..eaa1c1806f 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -23,6 +23,7 @@ import akka.util.duration._ import spark.{Logging, Utils} import spark.util.AkkaUtils import spark.deploy._ +import spark.metrics.MetricsSystem import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.text.SimpleDateFormat import java.util.Date @@ -41,7 +42,7 @@ private[spark] class Worker( memory: Int, masterUrl: String, workDirPath: String = null) - extends Actor with Logging with WorkerInstrumentation { + extends Actor with Logging { Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -67,6 +68,8 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + val workerInstrumentation = new WorkerInstrumentation(this) + def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -99,7 +102,8 @@ private[spark] class Worker( connectToMaster() startWebUi() - initialize(this) + Worker.metricsSystem.registerSource(workerInstrumentation) + Worker.metricsSystem.start() } def connectToMaster() { @@ -182,11 +186,13 @@ private[spark] class Worker( executors.values.foreach(_.kill()) webUi.stop() - uninitialize() + Worker.metricsSystem.stop() } } private[spark] object Worker { + private val metricsSystem = MetricsSystem.createMetricsSystem("worker") + def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 2f725300b5..37fd154859 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -1,58 +1,89 @@ package spark.deploy.worker -import com.codahale.metrics.{JmxReporter, Gauge, MetricRegistry} +import com.codahale.metrics.{Gauge, Metric} -import spark.metrics.AbstractInstrumentation +import java.util.{Map, HashMap => JHashMap} -private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { - var workerInst: Option[Worker] = None - val metricRegistry = new MetricRegistry() +import spark.metrics.source.Source + +private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { + val className = classOf[Worker].getName() - override def registryHandler = metricRegistry - - override def instance = "worker" - - def initialize(worker: Worker) { - workerInst = Some(worker) + override def sourceName = "worker" - // Register all the sources - registerSources() + override def getMetrics: Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] - // Register and start all the sinks - registerSinks() + // Gauge for executors number + gauges.put(className + ".executor.number", new Gauge[Int]{ + override def getValue: Int = worker.executors.size + }) + + gauges.put(className + ".core_used.number", new Gauge[Int]{ + override def getValue: Int = worker.coresUsed + }) + + gauges.put(className + ".mem_used.MBytes", new Gauge[Int]{ + override def getValue: Int = worker.memoryUsed + }) + + gauges.put(className + ".core_free.number", new Gauge[Int]{ + override def getValue: Int = worker.coresFree + }) + + gauges.put(className + ".mem_free.MBytes", new Gauge[Int]{ + override def getValue: Int = worker.memoryFree + }) + + gauges } - - def uninitialize() { - unregisterSinks() - } - - // Gauge for executors number - metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) - }) - - // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) - }) - - // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) - }) -} \ No newline at end of file +} +//private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { +// var workerInst: Option[Worker] = None +// val metricRegistry = new MetricRegistry() +// +// override def registryHandler = metricRegistry +// +// override def instance = "worker" +// +// def initialize(worker: Worker) { +// workerInst = Some(worker) +// +// registerSources() +// registerSinks() +// } +// +// def uninitialize() { +// unregisterSinks() +// } +// +// // Gauge for executors number +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) +// }) +// +// // Gauge for cores used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) +// }) +// +// // Gauge for memory used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) +// }) +// +// // Gauge for cores free of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) +// }) +// +// // Gauge for memory used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) +// }) +//} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala similarity index 64% rename from core/src/main/scala/spark/metrics/AbstractInstrumentation.scala rename to core/src/main/scala/spark/metrics/MetricsSystem.scala index 9cae1e0220..ea1bc490b5 100644 --- a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -2,7 +2,7 @@ package spark.metrics import scala.collection.mutable -import com.codahale.metrics.{JmxReporter, MetricRegistry} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit @@ -11,70 +11,76 @@ import spark.Logging import spark.metrics.sink._ import spark.metrics.source._ -private [spark] trait AbstractInstrumentation extends Logging { +private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - // Get MetricRegistry handler - def registryHandler: MetricRegistry - // Get the instance name - def instance: String - val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] + def start() { + registerSources() + registerSinks() + } + + def stop() { + sinks.foreach(_.stop) + } + + def registerSource(source: Source) { + sources += source + MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + } + def registerSources() { val instConfig = metricsConfig.getInstance(instance) - val sourceConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SOURCE_REGEX) + val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - // Register all the sources + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { - val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) - .newInstance(registryHandler) + val source = Class.forName(classPath).newInstance() sources += source.asInstanceOf[Source] + MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) } } - sources.foreach(_.registerSource) } def registerSinks() { val instConfig = metricsConfig.getInstance(instance) - val sinkConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SINK_REGEX) + val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) // Register JMX sink as a default sink - sinks += new JmxSink(registryHandler) + sinks += new JmxSink(MetricsSystem.registry) // Register other sinks according to conf sinkConfigs.foreach { kv => - val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { - AbstractInstrumentation.DEFAULT_SINKS(kv._1) + val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { + MetricsSystem.DEFAULT_SINKS(kv._1) } else { // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") } try { val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) - .newInstance(kv._2, registryHandler) + .newInstance(kv._2, MetricsSystem.registry) sinks += sink.asInstanceOf[Sink] } catch { case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.registerSink) - } - - def unregisterSinks() { - sinks.foreach(_.unregisterSink) + sinks.foreach(_.start) } } -object AbstractInstrumentation { +private[spark] object MetricsSystem { + val registry = new MetricRegistry() + val DEFAULT_SINKS = Map( "console" -> "spark.metrics.sink.ConsoleSink", "csv" -> "spark.metrics.sink.CsvSink") @@ -88,4 +94,6 @@ object AbstractInstrumentation { "minute" -> TimeUnit.MINUTES, "hour" -> TimeUnit.HOURS, "day" -> TimeUnit.DAYS) + + def createMetricsSystem(instance: String) = new MetricsSystem(instance) } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 5426af8c4c..b49a211fb3 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -5,7 +5,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{ConsoleReporter, MetricRegistry} -import spark.metrics.AbstractInstrumentation +import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { @@ -14,13 +14,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend } val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { - case Some(s) => AbstractInstrumentation.timeUnits(s) - case None => AbstractInstrumentation.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + case Some(s) => MetricsSystem.timeUnits(s) + case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } var reporter: ConsoleReporter = _ - override def registerSink() { + override def start() { reporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) @@ -29,7 +29,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend reporter.start(pollPeriod, pollUnit) } - override def unregisterSink() { + override def stop() { reporter.stop() } } diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 1d663f6cff..3f572c8e05 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -6,7 +6,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{CsvReporter, MetricRegistry} -import spark.metrics.AbstractInstrumentation +import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { @@ -15,8 +15,8 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si } val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { - case Some(s) => AbstractInstrumentation.timeUnits(s) - case None => AbstractInstrumentation.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + case Some(s) => MetricsSystem.timeUnits(s) + case None => MetricsSystem.timeUnits(CsvSink.CSV_DEFAULT_UNIT) } val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { @@ -26,7 +26,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si var reporter: CsvReporter = _ - override def registerSink() { + override def start() { reporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) @@ -36,7 +36,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si reporter.start(pollPeriod, pollUnit) } - override def unregisterSink() { + override def stop() { reporter.stop() } } diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index 56e5677700..e223dc26e9 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -5,12 +5,12 @@ import com.codahale.metrics.{JmxReporter, MetricRegistry} class JmxSink(registry: MetricRegistry) extends Sink { var reporter: JmxReporter = _ - override def registerSink() { + override def start() { reporter = JmxReporter.forRegistry(registry).build() reporter.start() } - override def unregisterSink() { + override def stop() { reporter.stop() } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 26052b7231..9fef894fde 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -1,7 +1,7 @@ package spark.metrics.sink trait Sink { - def registerSink: Unit + def start: Unit - def unregisterSink: Unit + def stop: Unit } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 8f6bf48843..7a7c1b6ffb 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -1,17 +1,23 @@ package spark.metrics.source -import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} +import java.util.{Map, HashMap => JHashMap} -class JvmSource(registry: MetricRegistry) extends Source { - // Initialize memory usage gauge for jvm - val memUsageMetricSet = new MemoryUsageGaugeSet - - // Initialize garbage collection usage gauge for jvm - val gcMetricSet = new GarbageCollectorMetricSet - - override def registerSource() { - registry.registerAll(memUsageMetricSet) - registry.registerAll(gcMetricSet) +import com.codahale.metrics.Metric +import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} + +class JvmSource extends Source { + override def sourceName = "jvm" + + override def getMetrics(): Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] + + import scala.collection.JavaConversions._ + val gcMetricSet = new GarbageCollectorMetricSet + gcMetricSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) + + val memGaugeSet = new MemoryUsageGaugeSet + memGaugeSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) + + gauges } } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index 35cfe0c8ff..edd59de46a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,5 +1,7 @@ package spark.metrics.source -trait Source { - def registerSource: Unit +import com.codahale.metrics.MetricSet + +trait Source extends MetricSet { + def sourceName: String } \ No newline at end of file From 4d6dd67fa1f8f031f1ef46a442cec733fa3b1a7a Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 28 Jun 2013 02:26:55 +0800 Subject: [PATCH 131/221] refactor metrics system 1.change source abstract class to support MetricRegistry 2.change master/work/jvm source class --- .../deploy/master/MasterInstrumentation.scala | 44 ++++----- .../deploy/worker/WorkerInstrumentation.scala | 98 +++++-------------- .../scala/spark/metrics/MetricsSystem.scala | 23 ++--- .../spark/metrics/sink/ConsoleSink.scala | 11 +-- .../scala/spark/metrics/sink/CsvSink.scala | 9 +- .../scala/spark/metrics/sink/JmxSink.scala | 5 +- .../spark/metrics/source/JvmSource.scala | 22 ++--- .../scala/spark/metrics/source/Source.scala | 6 +- 8 files changed, 84 insertions(+), 134 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 5ea9a90319..46c90b94d2 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -4,32 +4,32 @@ import java.util.{Map, HashMap => JHashMap} import com.codahale.metrics.{Gauge, Metric} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} + import spark.metrics.source.Source +import spark.Logging private[spark] class MasterInstrumentation(val master: Master) extends Source { val className = classOf[Master].getName() val instrumentationName = "master" - - override def sourceName = instrumentationName - - override def getMetrics(): Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - // Gauge for worker numbers in cluster - gauges.put(className + ".workers.number", new Gauge[Int] { + val metricRegistry = new MetricRegistry() + val sourceName = instrumentationName + + metricRegistry.register(MetricRegistry.name("workers","number"), + new Gauge[Int] { override def getValue: Int = master.workers.size - }) - - // Gauge for application numbers in cluster - gauges.put(className + ".apps.number", new Gauge[Int] { - override def getValue: Int = master.apps.size - }) - - // Gauge for waiting application numbers in cluster - gauges.put(className + ".waiting_apps.number", new Gauge[Int] { + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name("apps", "number"), + new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), + new Gauge[Int] { override def getValue: Int = master.waitingApps.size - }) - - gauges - } -} \ No newline at end of file + }) + +} diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 37fd154859..5ce29cf04c 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -4,86 +4,42 @@ import com.codahale.metrics.{Gauge, Metric} import java.util.{Map, HashMap => JHashMap} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import spark.metrics.source.Source private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { val className = classOf[Worker].getName() - override def sourceName = "worker" - - override def getMetrics: Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - // Gauge for executors number - gauges.put(className + ".executor.number", new Gauge[Int]{ + val sourceName = "worker" + + val metricRegistry = new MetricRegistry() + + metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + new Gauge[Int] { override def getValue: Int = worker.executors.size - }) - - gauges.put(className + ".core_used.number", new Gauge[Int]{ + }) + + // Gauge for cores used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + new Gauge[Int] { override def getValue: Int = worker.coresUsed - }) - - gauges.put(className + ".mem_used.MBytes", new Gauge[Int]{ + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), + new Gauge[Int] { override def getValue: Int = worker.memoryUsed - }) - - gauges.put(className + ".core_free.number", new Gauge[Int]{ + }) + + // Gauge for cores free of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + new Gauge[Int] { override def getValue: Int = worker.coresFree - }) - - gauges.put(className + ".mem_free.MBytes", new Gauge[Int]{ + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + new Gauge[Int] { override def getValue: Int = worker.memoryFree }) - - gauges - } } -//private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { -// var workerInst: Option[Worker] = None -// val metricRegistry = new MetricRegistry() -// -// override def registryHandler = metricRegistry -// -// override def instance = "worker" -// -// def initialize(worker: Worker) { -// workerInst = Some(worker) -// -// registerSources() -// registerSinks() -// } -// -// def uninitialize() { -// unregisterSinks() -// } -// -// // Gauge for executors number -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) -// }) -// -// // Gauge for cores used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) -// }) -// -// // Gauge for memory used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) -// }) -// -// // Gauge for cores free of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) -// }) -// -// // Gauge for memory used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) -// }) -//} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index ea1bc490b5..a23ccd2692 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -5,6 +5,7 @@ import scala.collection.mutable import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties +//import java.util._ import java.util.concurrent.TimeUnit import spark.Logging @@ -19,10 +20,13 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] + var registry = new MetricRegistry() + + registerSources() + registerSinks() def start() { - registerSources() - registerSinks() + sinks.foreach(_.start) } def stop() { @@ -31,20 +35,20 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + registry.register(source.sourceName,source.metricRegistry) } def registerSources() { val instConfig = metricsConfig.getInstance(instance) val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { val source = Class.forName(classPath).newInstance() sources += source.asInstanceOf[Source] - MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + registerSource(source.asInstanceOf[Source]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) } @@ -56,7 +60,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) // Register JMX sink as a default sink - sinks += new JmxSink(MetricsSystem.registry) + sinks += new JmxSink(registry) // Register other sinks according to conf sinkConfigs.foreach { kv => @@ -68,19 +72,16 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } try { val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) - .newInstance(kv._2, MetricsSystem.registry) + .newInstance(kv._2, registry) sinks += sink.asInstanceOf[Sink] } catch { case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.start) } } private[spark] object MetricsSystem { - val registry = new MetricRegistry() - val DEFAULT_SINKS = Map( "console" -> "spark.metrics.sink.ConsoleSink", "csv" -> "spark.metrics.sink.CsvSink") @@ -96,4 +97,4 @@ private[spark] object MetricsSystem { "day" -> TimeUnit.DAYS) def createMetricsSystem(instance: String) = new MetricsSystem(instance) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index b49a211fb3..9cd17556fa 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -18,14 +18,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } - var reporter: ConsoleReporter = _ - - override def start() { - reporter = ConsoleReporter.forRegistry(registry) + var reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() - + + + override def start() { reporter.start(pollPeriod, pollUnit) } @@ -40,4 +39,4 @@ object ConsoleSink { val CONSOLE_KEY_PERIOD = "period" val CONSOLE_KEY_UNIT = "unit" -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 3f572c8e05..62e51be0dc 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -24,15 +24,14 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si case None => CsvSink.CSV_DEFAULT_DIR } - var reporter: CsvReporter = _ - - override def start() { - reporter = CsvReporter.forRegistry(registry) + var reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build(new File(pollDir)) - + + + override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index e223dc26e9..98b55f7b7f 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -3,10 +3,9 @@ package spark.metrics.sink import com.codahale.metrics.{JmxReporter, MetricRegistry} class JmxSink(registry: MetricRegistry) extends Sink { - var reporter: JmxReporter = _ + var reporter: JmxReporter = JmxReporter.forRegistry(registry).build() override def start() { - reporter = JmxReporter.forRegistry(registry).build() reporter.start() } @@ -14,4 +13,4 @@ class JmxSink(registry: MetricRegistry) extends Sink { reporter.stop() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 7a7c1b6ffb..13270dae3c 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -2,22 +2,16 @@ package spark.metrics.source import java.util.{Map, HashMap => JHashMap} -import com.codahale.metrics.Metric +import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} class JvmSource extends Source { - override def sourceName = "jvm" - - override def getMetrics(): Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - import scala.collection.JavaConversions._ + val sourceName = "jvm" + val metricRegistry = new MetricRegistry() + val gcMetricSet = new GarbageCollectorMetricSet - gcMetricSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) - val memGaugeSet = new MemoryUsageGaugeSet - memGaugeSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) - - gauges - } -} \ No newline at end of file + + metricRegistry.registerAll(gcMetricSet) + metricRegistry.registerAll(memGaugeSet) +} diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index edd59de46a..17cbe2f85a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,7 +1,9 @@ package spark.metrics.source import com.codahale.metrics.MetricSet +import com.codahale.metrics.MetricRegistry -trait Source extends MetricSet { +trait Source { def sourceName: String -} \ No newline at end of file + def metricRegistry: MetricRegistry +} From 7fb574bf666661fdf8a786de779f85efe2f15f0c Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:14:30 +0800 Subject: [PATCH 132/221] Code clean and remarshal --- .../scala/spark/deploy/master/Master.scala | 9 +++++---- .../deploy/master/MasterInstrumentation.scala | 11 ++--------- .../deploy/worker/WorkerInstrumentation.scala | 18 ++++++------------ .../scala/spark/metrics/MetricsSystem.scala | 5 ++--- .../scala/spark/metrics/sink/ConsoleSink.scala | 1 - .../scala/spark/metrics/sink/CsvSink.scala | 1 - .../main/scala/spark/metrics/sink/Sink.scala | 1 - 7 files changed, 15 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index e44f5e3168..cc0b2d4295 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -78,6 +78,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) Master.metricsSystem.registerSource(masterInstrumentation) + Master.metricsSystem.start() } override def postStop() { @@ -321,22 +322,22 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } + + override def postStop() { + Master.metricsSystem.stop() + } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r - private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) - - metricsSystem.start() actorSystem.awaitTermination() - metricsSystem.stop() } /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 46c90b94d2..61a561c955 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -1,19 +1,12 @@ package spark.deploy.master -import java.util.{Map, HashMap => JHashMap} - -import com.codahale.metrics.{Gauge, Metric} - -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} +import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -import spark.Logging private[spark] class MasterInstrumentation(val master: Master) extends Source { - val className = classOf[Master].getName() - val instrumentationName = "master" val metricRegistry = new MetricRegistry() - val sourceName = instrumentationName + val sourceName = "master" metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 5ce29cf04c..94c20a98c1 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -1,44 +1,38 @@ package spark.deploy.worker -import com.codahale.metrics.{Gauge, Metric} +import com.codahale.metrics.{Gauge, MetricRegistry} -import java.util.{Map, HashMap => JHashMap} - -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import spark.metrics.source.Source private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { - val className = classOf[Worker].getName() - val sourceName = "worker" - val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { override def getValue: Int = worker.executors.size }) // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { override def getValue: Int = worker.coresUsed }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), + metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryUsed }) // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { override def getValue: Int = worker.coresFree }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryFree }) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index a23ccd2692..5bfdc00eaf 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -5,7 +5,6 @@ import scala.collection.mutable import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties -//import java.util._ import java.util.concurrent.TimeUnit import spark.Logging @@ -20,7 +19,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] - var registry = new MetricRegistry() + val registry = new MetricRegistry() registerSources() registerSinks() @@ -35,7 +34,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - registry.register(source.sourceName,source.metricRegistry) + registry.register(source.sourceName, source.metricRegistry) } def registerSources() { diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 9cd17556fa..e2e4197d1d 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -23,7 +23,6 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend .convertRatesTo(TimeUnit.SECONDS) .build() - override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 62e51be0dc..c2d645331c 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -30,7 +30,6 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si .convertRatesTo(TimeUnit.SECONDS) .build(new File(pollDir)) - override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 9fef894fde..3ffdcbdaba 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -2,6 +2,5 @@ package spark.metrics.sink trait Sink { def start: Unit - def stop: Unit } \ No newline at end of file From 576528f0f916cc0434972cbbc9321e4dcec45628 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:25:48 +0800 Subject: [PATCH 133/221] Add dependency of Codahale's metrics library --- project/SparkBuild.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f3f67b57c8..825319d3de 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -180,6 +180,8 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "com.codahale.metrics" % "metrics-core" % "3.0.0", + "com.codahale.metrics" % "metrics-jvm" % "3.0.0" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { From 871bc1687eaeb59df24b4778c5992a5f7f105cc8 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:46:24 +0800 Subject: [PATCH 134/221] Add Executor instrumentation --- .../executor/ExecutorInstrumentation.scala | 35 +++++++++++++++++++ .../spark/executor/MesosExecutorBackend.scala | 10 +++++- .../executor/StandaloneExecutorBackend.scala | 12 +++++++ 3 files changed, 56 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/spark/executor/ExecutorInstrumentation.scala diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala new file mode 100644 index 0000000000..80aadb66b0 --- /dev/null +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -0,0 +1,35 @@ +package spark.executor + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import spark.metrics.source.Source + +class ExecutorInstrumentation(val executor: Option[Executor]) extends Source{ + val metricRegistry = new MetricRegistry() + val sourceName = "executor" + + // Gauge for executor thread pool's actively executing task counts + metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getActiveCount()).getOrElse(0) + }) + + // Gauge for executor thread pool's approximate total number of tasks that have been completed + metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), + new Gauge[Long] { + override def getValue: Long = executor.map(_.threadPool.getCompletedTaskCount()).getOrElse(0) + }) + + // Gauge for executor thread pool's current number of threads + metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getPoolSize()).getOrElse(0) + }) + + // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getMaximumPoolSize()).getOrElse(0) + }) + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 4961c42fad..8b6ab0c391 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -24,6 +24,7 @@ import spark.TaskState.TaskState import com.google.protobuf.ByteString import spark.{Utils, Logging} import spark.TaskState +import spark.metrics.MetricsSystem private[spark] class MesosExecutorBackend extends MesosExecutor @@ -32,6 +33,9 @@ private[spark] class MesosExecutorBackend var executor: Executor = null var driver: ExecutorDriver = null + + val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) + MesosExecutorBackend.metricsSystem.start() override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() @@ -79,13 +83,17 @@ private[spark] class MesosExecutorBackend override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {} - override def shutdown(d: ExecutorDriver) {} + override def shutdown(d: ExecutorDriver) { + MesosExecutorBackend.metricsSystem.stop() + } } /** * Entry point for Mesos executor. */ private[spark] object MesosExecutorBackend { + private val metricsSystem = MetricsSystem.createMetricsSystem("executor") + def main(args: Array[String]) { MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index f4003da732..6ef74cd2ff 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -24,6 +24,7 @@ import spark.util.AkkaUtils import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} +import spark.metrics.MetricsSystem import spark.scheduler.cluster._ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask @@ -45,6 +46,8 @@ private[spark] class StandaloneExecutorBackend( var executor: Executor = null var driver: ActorRef = null + + val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) override def preStart() { logInfo("Connecting to driver: " + driverUrl) @@ -52,6 +55,9 @@ private[spark] class StandaloneExecutorBackend( driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(driver) // Doesn't work with remote actors, but useful for testing + + StandaloneExecutorBackend.metricsSystem.registerSource(executorInstrumentation) + StandaloneExecutorBackend.metricsSystem.start() } override def receive = { @@ -81,9 +87,15 @@ private[spark] class StandaloneExecutorBackend( override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { driver ! StatusUpdate(executorId, taskId, state, data) } + + override def postStop() { + StandaloneExecutorBackend.metricsSystem.stop() + } } private[spark] object StandaloneExecutorBackend { + private val metricsSystem = MetricsSystem.createMetricsSystem("executor") + def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) } From 5ce5dc9fcd7acf5c58dd3d456a629b01d57514e4 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:48:21 +0800 Subject: [PATCH 135/221] Add default properties to deal with no configure file situation --- .../scala/spark/metrics/MetricsConfig.scala | 28 ++++++++++++------- .../scala/spark/metrics/MetricsSystem.scala | 9 ++---- .../spark/metrics/sink/ConsoleSink.scala | 6 ++-- .../scala/spark/metrics/sink/CsvSink.scala | 6 ++-- .../scala/spark/metrics/sink/JmxSink.scala | 6 ++-- 5 files changed, 30 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index be4f670918..7405192058 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -1,20 +1,25 @@ package spark.metrics import java.util.Properties -import java.io.FileInputStream +import java.io.{File, FileInputStream} import scala.collection.mutable import scala.util.matching.Regex private [spark] class MetricsConfig(val configFile: String) { val properties = new Properties() - var fis: FileInputStream = _ + // Add default properties in case there's no properties file + MetricsConfig.setDefaultProperties(properties) - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() + val confFile = new File(configFile) + if (confFile.exists()) { + var fis: FileInputStream = null + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } } val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) @@ -35,11 +40,15 @@ private [spark] class MetricsConfig(val configFile: String) { } } -object MetricsConfig { - val DEFAULT_CONFIG_FILE = "conf/metrics.properties" +private[spark] object MetricsConfig { val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + def setDefaultProperties(prop: Properties) { + prop.setProperty("*.sink.jmx.enabled", "default") + prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + } + def subProperties(prop: Properties, regex: Regex) = { val subProperties = new mutable.HashMap[String, Properties] @@ -48,7 +57,6 @@ object MetricsConfig { if (regex.findPrefixOf(kv._1) != None) { val regex(prefix, suffix) = kv._1 subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) - println(">>>>>subProperties added " + prefix + " " + suffix + " " + kv._2) } } diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 5bfdc00eaf..6e448cb2a5 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -14,7 +14,7 @@ import spark.metrics.source._ private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) + val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] @@ -58,9 +58,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - // Register JMX sink as a default sink - sinks += new JmxSink(registry) - // Register other sinks according to conf sinkConfigs.foreach { kv => val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { @@ -81,9 +78,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } private[spark] object MetricsSystem { - val DEFAULT_SINKS = Map( - "console" -> "spark.metrics.sink.ConsoleSink", - "csv" -> "spark.metrics.sink.CsvSink") + val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index e2e4197d1d..d7b7a9e501 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -1,10 +1,10 @@ package spark.metrics.sink +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + import java.util.Properties import java.util.concurrent.TimeUnit -import com.codahale.metrics.{ConsoleReporter, MetricRegistry} - import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { @@ -18,7 +18,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } - var reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index c2d645331c..e6c5bffd3c 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -1,11 +1,11 @@ package spark.metrics.sink +import com.codahale.metrics.{CsvReporter, MetricRegistry} + import java.io.File import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit -import com.codahale.metrics.{CsvReporter, MetricRegistry} - import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { @@ -24,7 +24,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si case None => CsvSink.CSV_DEFAULT_DIR } - var reporter: CsvReporter = CsvReporter.forRegistry(registry) + val reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index 98b55f7b7f..f097a631c0 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -1,9 +1,11 @@ package spark.metrics.sink +import java.util.Properties + import com.codahale.metrics.{JmxReporter, MetricRegistry} -class JmxSink(registry: MetricRegistry) extends Sink { - var reporter: JmxReporter = JmxReporter.forRegistry(registry).build() +class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() override def start() { reporter.start() From e080588f7396d9612ea5d909e59f2364c139103a Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:04:06 +0800 Subject: [PATCH 136/221] Add metrics system unit test --- .../spark/metrics/MetricsConfigSuite.scala | 92 +++++++++++++++++++ .../spark/metrics/MetricsSystemSuite.scala | 65 +++++++++++++ 2 files changed, 157 insertions(+) create mode 100644 core/src/test/scala/spark/metrics/MetricsConfigSuite.scala create mode 100644 core/src/test/scala/spark/metrics/MetricsSystemSuite.scala diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala new file mode 100644 index 0000000000..0c7142c418 --- /dev/null +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -0,0 +1,92 @@ +package spark.metrics + +import java.util.Properties +import java.io.{File, FileOutputStream} + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import spark.metrics._ + +class MetricsConfigSuite extends FunSuite with BeforeAndAfter { + var filePath: String = _ + + before { + val prop = new Properties() + + prop.setProperty("*.sink.console.period", "10") + prop.setProperty("*.sink.console.unit", "second") + prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + prop.setProperty("master.sink.console.period", "20") + prop.setProperty("master.sink.console.unit", "minute") + + val dir = new File("/tmp") + filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { + "/tmp/test_metrics.properties" + } else { + "./test_metrics.properties" + } + + val os = new FileOutputStream(new File(filePath)) + prop.store(os, "for test") + os.close() + } + + test("MetricsConfig with default properties") { + val conf = new MetricsConfig("dummy-file") + assert(conf.properties.size() === 2) + assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") + assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") + assert(conf.properties.getProperty("test-for-dummy") === null) + + val property = conf.getInstance("random") + assert(property.size() === 2) + assert(property.getProperty("sink.jmx.enabled") === "default") + assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") + } + + test("MetricsConfig with properties set") { + val conf = new MetricsConfig(filePath) + + val masterProp = conf.getInstance("master") + assert(masterProp.size() === 4) + assert(masterProp.getProperty("sink.console.period") === "20") + assert(masterProp.getProperty("sink.console.unit") === "minute") + assert(masterProp.getProperty("sink.jmx.enabled") === "default") + assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") + + val workerProp = conf.getInstance("worker") + assert(workerProp.size() === 4) + assert(workerProp.getProperty("sink.console.period") === "10") + assert(workerProp.getProperty("sink.console.unit") === "second") + } + + test("MetricsConfig with subProperties") { + val conf = new MetricsConfig(filePath) + + val propCategories = conf.propertyCategories + assert(propCategories.size === 2) + + val masterProp = conf.getInstance("master") + val sourceProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) + assert(sourceProps.size === 1) + assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") + + val sinkProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SINK_REGEX) + assert(sinkProps.size === 2) + assert(sinkProps.contains("console")) + assert(sinkProps.contains("jmx")) + + val consoleProps = sinkProps("console") + assert(consoleProps.size() === 2) + + val jmxProps = sinkProps("jmx") + assert(jmxProps.size() === 1) + } + + after { + val file = new File(filePath) + if (file.exists()) { + file.delete() + } + } +} \ No newline at end of file diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala new file mode 100644 index 0000000000..5e8f8fcf80 --- /dev/null +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -0,0 +1,65 @@ +package spark.metrics + +import java.util.Properties +import java.io.{File, FileOutputStream} + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import spark.metrics._ + +class MetricsSystemSuite extends FunSuite with BeforeAndAfter { + var filePath: String = _ + + before { + val props = new Properties() + props.setProperty("*.sink.console.period", "10") + props.setProperty("*.sink.console.unit", "second") + props.setProperty("test.sink.console.class", "spark.metrics.sink.ConsoleSink") + props.setProperty("test.sink.dummy.class", "spark.metrics.sink.DummySink") + props.setProperty("test.source.dummy.class", "spark.metrics.source.DummySource") + props.setProperty("test.sink.console.period", "20") + props.setProperty("test.sink.console.unit", "minute") + + val dir = new File("/tmp") + filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { + "/tmp/test_metrics.properties" + } else { + "./test_metrics.properties" + } + + val os = new FileOutputStream(new File(filePath)) + props.store(os, "for test") + os.close() + System.setProperty("spark.metrics.conf.file", filePath) + } + + test("MetricsSystem with default config") { + val metricsSystem = MetricsSystem.createMetricsSystem("default") + val sources = metricsSystem.sources + val sinks = metricsSystem.sinks + + assert(sources.length === 1) + assert(sinks.length === 1) + assert(sources(0).sourceName === "jvm") + } + + test("MetricsSystem with sources add") { + val metricsSystem = MetricsSystem.createMetricsSystem("test") + val sources = metricsSystem.sources + val sinks = metricsSystem.sinks + + assert(sources.length === 1) + assert(sinks.length === 2) + + val source = new spark.deploy.master.MasterInstrumentation(null) + metricsSystem.registerSource(source) + assert(sources.length === 2) + } + + after { + val file = new File(filePath) + if (file.exists()) { + file.delete() + } + } +} \ No newline at end of file From e9ac88754d4c5d58aedd4de8768787300b15eada Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:05:18 +0800 Subject: [PATCH 137/221] Remove twice add Source bug and code clean --- .../src/main/scala/spark/metrics/MetricsSystem.scala | 10 ++++------ .../main/scala/spark/metrics/source/JvmSource.scala | 12 ++++++------ 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 6e448cb2a5..bf4487e0fc 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -1,15 +1,15 @@ package spark.metrics -import scala.collection.mutable - import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit +import scala.collection.mutable + import spark.Logging -import spark.metrics.sink._ -import spark.metrics.source._ +import spark.metrics.sink.Sink +import spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() @@ -46,7 +46,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val classPath = kv._2.getProperty("class") try { val source = Class.forName(classPath).newInstance() - sources += source.asInstanceOf[Source] registerSource(source.asInstanceOf[Source]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) @@ -58,7 +57,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - // Register other sinks according to conf sinkConfigs.foreach { kv => val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { MetricsSystem.DEFAULT_SINKS(kv._1) diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 13270dae3c..8f0870c1a0 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -1,17 +1,17 @@ package spark.metrics.source -import java.util.{Map, HashMap => JHashMap} - import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} +import java.util.{Map, HashMap => JHashMap} + class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() - val gcMetricSet = new GarbageCollectorMetricSet - val memGaugeSet = new MemoryUsageGaugeSet + val gcMetricSet = new GarbageCollectorMetricSet + val memGaugeSet = new MemoryUsageGaugeSet - metricRegistry.registerAll(gcMetricSet) - metricRegistry.registerAll(memGaugeSet) + metricRegistry.registerAll(gcMetricSet) + metricRegistry.registerAll(memGaugeSet) } From 7d2eada451686824bd467641bf1763e82011f2a6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 21:37:21 +0800 Subject: [PATCH 138/221] Add metrics source of DAGScheduler and blockManager Conflicts: core/src/main/scala/spark/SparkContext.scala core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkContext.scala | 13 +++++-- core/src/main/scala/spark/SparkEnv.scala | 6 ++++ .../scala/spark/scheduler/DAGScheduler.scala | 3 +- .../spark/scheduler/DAGSchedulerSource.scala | 31 ++++++++++++++++ .../spark/storage/BlockManagerSource.scala | 35 +++++++++++++++++++ 5 files changed, 84 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala create mode 100644 core/src/main/scala/spark/storage/BlockManagerSource.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 24ba605646..a6128a9f30 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -60,11 +60,11 @@ import org.apache.mesos.MesosNativeLibrary import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} +import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import spark.storage.{StorageStatus, StorageUtils, RDDInfo} +import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} @@ -270,6 +270,15 @@ class SparkContext( // Post init taskScheduler.postStartHook() + val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) + val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) + def initDriverMetrics() = { + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + } + + initDriverMetrics() + // Methods for creating RDDs /** Distribute a local Scala collection to form an RDD. */ diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index f2bdc11bdb..d34dafecc5 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -30,6 +30,7 @@ import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils import spark.api.python.PythonWorkerFactory +import spark.metrics._ /** @@ -53,6 +54,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, + val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -184,6 +186,9 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + val metricsSystem = MetricsSystem.createMetricsSystem("driver") + metricsSystem.start() + // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working // directory. @@ -213,6 +218,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, + metricsSystem, None) } } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 9b45fc2938..781e49bdec 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -30,7 +30,7 @@ import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialRe import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} - +import spark.metrics.MetricsSystem /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal @@ -126,7 +126,6 @@ class DAGScheduler( val resultStageToJob = new HashMap[Stage, ActiveJob] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) - // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala new file mode 100644 index 0000000000..57aa74512c --- /dev/null +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -0,0 +1,31 @@ +package spark.scheduler + +import com.codahale.metrics.{Gauge,MetricRegistry} + +import spark.metrics.source.Source + +private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "DAGScheduler" + + + metricRegistry.register(MetricRegistry.name("stage","failedStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.failed.size + }) + + metricRegistry.register(MetricRegistry.name("stage","runningStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.running.size + }) + + metricRegistry.register(MetricRegistry.name("stage","waitingStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.waiting.size + }) + + metricRegistry.register(MetricRegistry.name("job","allJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.nextRunId.get() + }) + + metricRegistry.register(MetricRegistry.name("job","ActiveJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.activeJobs.size + }) +} diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala new file mode 100644 index 0000000000..c0ce9259c8 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -0,0 +1,35 @@ +package spark.storage + +import com.codahale.metrics.{Gauge,MetricRegistry} + +import spark.metrics.source.Source +import spark.storage._ + +private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "BlockManager" + + metricRegistry.register(MetricRegistry.name("memory","maxMem"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + maxMem + } + }) + + metricRegistry.register(MetricRegistry.name("memory","remainingMem"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + remainingMem + } + }) + + metricRegistry.register(MetricRegistry.name("disk","diskSpaceUsed"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) + diskSpaceUsed + } + }) +} From 9cea0c28184c86625f8281eea2af77eff15acb73 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 22:22:29 +0800 Subject: [PATCH 139/221] Refactor metricsSystem unit test, add resource files. --- .../resources/test_metrics_config.properties | 6 ++++ .../resources/test_metrics_system.properties | 7 +++++ .../spark/metrics/MetricsConfigSuite.scala | 30 ++---------------- .../spark/metrics/MetricsSystemSuite.scala | 31 ++----------------- 4 files changed, 19 insertions(+), 55 deletions(-) create mode 100644 core/src/test/resources/test_metrics_config.properties create mode 100644 core/src/test/resources/test_metrics_system.properties diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties new file mode 100644 index 0000000000..2011940003 --- /dev/null +++ b/core/src/test/resources/test_metrics_config.properties @@ -0,0 +1,6 @@ +*.sink.console.period = 10 +*.sink.console.unit = second +*.source.jvm.class = spark.metrics.source.JvmSource +master.sink.console.period = 20 +master.sink.console.unit = minute + diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties new file mode 100644 index 0000000000..06afbc6625 --- /dev/null +++ b/core/src/test/resources/test_metrics_system.properties @@ -0,0 +1,7 @@ +*.sink.console.period = 10 +*.sink.console.unit = second +test.sink.console.class = spark.metrics.sink.ConsoleSink +test.sink.dummy.class = spark.metrics.sink.DummySink +test.source.dummy.class = spark.metrics.source.DummySource +test.sink.console.period = 20 +test.sink.console.unit = minute diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index 0c7142c418..f4c83cb644 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -11,24 +11,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ before { - val prop = new Properties() - - prop.setProperty("*.sink.console.period", "10") - prop.setProperty("*.sink.console.unit", "second") - prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") - prop.setProperty("master.sink.console.period", "20") - prop.setProperty("master.sink.console.unit", "minute") - - val dir = new File("/tmp") - filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { - "/tmp/test_metrics.properties" - } else { - "./test_metrics.properties" - } - - val os = new FileOutputStream(new File(filePath)) - prop.store(os, "for test") - os.close() + filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile() } test("MetricsConfig with default properties") { @@ -81,12 +64,5 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { val jmxProps = sinkProps("jmx") assert(jmxProps.size() === 1) - } - - after { - val file = new File(filePath) - if (file.exists()) { - file.delete() - } - } -} \ No newline at end of file + } +} diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 5e8f8fcf80..967be6ec47 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -11,25 +11,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ before { - val props = new Properties() - props.setProperty("*.sink.console.period", "10") - props.setProperty("*.sink.console.unit", "second") - props.setProperty("test.sink.console.class", "spark.metrics.sink.ConsoleSink") - props.setProperty("test.sink.dummy.class", "spark.metrics.sink.DummySink") - props.setProperty("test.source.dummy.class", "spark.metrics.source.DummySource") - props.setProperty("test.sink.console.period", "20") - props.setProperty("test.sink.console.unit", "minute") - - val dir = new File("/tmp") - filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { - "/tmp/test_metrics.properties" - } else { - "./test_metrics.properties" - } - - val os = new FileOutputStream(new File(filePath)) - props.store(os, "for test") - os.close() + filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf.file", filePath) } @@ -54,12 +36,5 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val source = new spark.deploy.master.MasterInstrumentation(null) metricsSystem.registerSource(source) assert(sources.length === 2) - } - - after { - val file = new File(filePath) - if (file.exists()) { - file.delete() - } - } -} \ No newline at end of file + } +} From 5f8802c1fb106cc04c30b6aca0a6ce98fa5c0e15 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 2 Jul 2013 10:41:43 +0800 Subject: [PATCH 140/221] Register and init metricsSystem in SparkContext Conflicts: core/src/main/scala/spark/SparkContext.scala core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkContext.scala | 10 ++++++++-- core/src/main/scala/spark/SparkEnv.scala | 4 ---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index a6128a9f30..1255d0c72e 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -67,6 +67,9 @@ import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} +import spark.metrics._ + +import scala.util.DynamicVariable /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -272,9 +275,12 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) + val metricsSystem = MetricsSystem.createMetricsSystem("driver") + def initDriverMetrics() = { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + metricsSystem.registerSource(dagSchedulerSource) + metricsSystem.registerSource(blockManagerSource) + metricsSystem.start() } initDriverMetrics() diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index d34dafecc5..125dc55bd8 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -54,7 +54,6 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -186,8 +185,6 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) - val metricsSystem = MetricsSystem.createMetricsSystem("driver") - metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working @@ -218,7 +215,6 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - metricsSystem, None) } } From 1daff54b2ed92d0bcee7030d7d3ab5c274f80d2f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:28:32 +0800 Subject: [PATCH 141/221] Change Executor MetricsSystem initialize code to SparkEnv --- core/src/main/scala/spark/SparkContext.scala | 6 ++---- core/src/main/scala/spark/SparkEnv.scala | 9 +++++++++ core/src/main/scala/spark/executor/Executor.scala | 6 +++++- .../spark/executor/ExecutorInstrumentation.scala | 10 +++++----- .../scala/spark/executor/MesosExecutorBackend.scala | 10 +--------- .../spark/executor/StandaloneExecutorBackend.scala | 12 ------------ 6 files changed, 22 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1255d0c72e..f1d9d5e442 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -275,12 +275,10 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - val metricsSystem = MetricsSystem.createMetricsSystem("driver") def initDriverMetrics() = { - metricsSystem.registerSource(dagSchedulerSource) - metricsSystem.registerSource(blockManagerSource) - metricsSystem.start() + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) } initDriverMetrics() diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 125dc55bd8..7b3dc69348 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -54,6 +54,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, + val metricsSystem: metricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -69,6 +70,7 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() + metricsSystem.stop() actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release @@ -185,6 +187,12 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + val metricsSystem = if (isDriver) { + MetricsSystem.createMetricsSystem("driver") + } else { + MetricsSystem.createMetricsSystem("executor") + } + metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working @@ -215,6 +223,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, + metricsSystem, None) } } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 2e81151882..7179ed84a8 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -86,10 +86,14 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } } ) + + val executorInstrumentation = new ExecutorInstrumentation(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) - SparkEnv.set(env) + SparkEnv.set(env) + env.metricsSystem.registerSource(executorInstrumentation) + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala index 80aadb66b0..ebbcbee742 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -4,32 +4,32 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Option[Executor]) extends Source{ +class ExecutorInstrumentation(val executor: Executor) extends Source{ val metricRegistry = new MetricRegistry() val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getActiveCount()).getOrElse(0) + override def getValue: Int = executor.threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { - override def getValue: Long = executor.map(_.threadPool.getCompletedTaskCount()).getOrElse(0) + override def getValue: Long = executor.threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getPoolSize()).getOrElse(0) + override def getValue: Int = executor.threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getMaximumPoolSize()).getOrElse(0) + override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) } \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 8b6ab0c391..4961c42fad 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -24,7 +24,6 @@ import spark.TaskState.TaskState import com.google.protobuf.ByteString import spark.{Utils, Logging} import spark.TaskState -import spark.metrics.MetricsSystem private[spark] class MesosExecutorBackend extends MesosExecutor @@ -33,9 +32,6 @@ private[spark] class MesosExecutorBackend var executor: Executor = null var driver: ExecutorDriver = null - - val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) - MesosExecutorBackend.metricsSystem.start() override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() @@ -83,17 +79,13 @@ private[spark] class MesosExecutorBackend override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {} - override def shutdown(d: ExecutorDriver) { - MesosExecutorBackend.metricsSystem.stop() - } + override def shutdown(d: ExecutorDriver) {} } /** * Entry point for Mesos executor. */ private[spark] object MesosExecutorBackend { - private val metricsSystem = MetricsSystem.createMetricsSystem("executor") - def main(args: Array[String]) { MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 6ef74cd2ff..f4003da732 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -24,7 +24,6 @@ import spark.util.AkkaUtils import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} -import spark.metrics.MetricsSystem import spark.scheduler.cluster._ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask @@ -46,8 +45,6 @@ private[spark] class StandaloneExecutorBackend( var executor: Executor = null var driver: ActorRef = null - - val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) override def preStart() { logInfo("Connecting to driver: " + driverUrl) @@ -55,9 +52,6 @@ private[spark] class StandaloneExecutorBackend( driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(driver) // Doesn't work with remote actors, but useful for testing - - StandaloneExecutorBackend.metricsSystem.registerSource(executorInstrumentation) - StandaloneExecutorBackend.metricsSystem.start() } override def receive = { @@ -87,15 +81,9 @@ private[spark] class StandaloneExecutorBackend( override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { driver ! StatusUpdate(executorId, taskId, state, data) } - - override def postStop() { - StandaloneExecutorBackend.metricsSystem.stop() - } } private[spark] object StandaloneExecutorBackend { - private val metricsSystem = MetricsSystem.createMetricsSystem("executor") - def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) } From a79f6077f0b3bc0110f6e8e15de3068f296f3b81 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:29:26 +0800 Subject: [PATCH 142/221] Add Maven metrics library dependency and code changes --- conf/metrics.properties | 4 ++-- core/pom.xml | 8 ++++++++ core/src/main/scala/spark/SparkEnv.scala | 3 ++- pom.xml | 8 ++++++++ 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/conf/metrics.properties b/conf/metrics.properties index 0bbb6b5229..2ea3aaf345 100644 --- a/conf/metrics.properties +++ b/conf/metrics.properties @@ -1,10 +1,10 @@ -# syntax: [prefix].[sink|source].[instance].[options] +# syntax: [instance].[sink|source].[name].[options] *.sink.console.period=10 *.sink.console.unit=second -master.source.jvm.class=spark.metrics.source.JvmSource +*.sink.console.class=spark.metrics.sink.ConsoleSink master.sink.console.period=10 diff --git a/core/pom.xml b/core/pom.xml index 6329b2fbd8..237d988712 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -108,6 +108,14 @@ log4j log4j + + com.codahale.metrics + metrics-core + + + com.codahale.metrics + metrics-jvm + org.apache.derby diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7b3dc69348..204049d927 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -24,6 +24,7 @@ import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider import spark.broadcast.BroadcastManager +import spark.metrics.MetricsSystem import spark.storage.BlockManager import spark.storage.BlockManagerMaster import spark.network.ConnectionManager @@ -54,7 +55,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: metricsSystem, + val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort diff --git a/pom.xml b/pom.xml index eb7bd7e9df..3fbd93c7dd 100644 --- a/pom.xml +++ b/pom.xml @@ -268,6 +268,14 @@ org.scala-lang scalap ${scala.version} + com.codahale.metrics + metrics-core + 3.0.0 + + + com.codahale.metrics + metrics-jvm + 3.0.0 From 5730193e0c8639b654f489c5956e31451d81b7db Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 13:48:09 +0800 Subject: [PATCH 143/221] Fix some typos --- core/src/main/scala/spark/SparkContext.scala | 2 -- core/src/main/scala/spark/SparkEnv.scala | 1 - core/src/main/scala/spark/metrics/sink/JmxSink.scala | 10 +++++----- .../main/scala/spark/metrics/source/JvmSource.scala | 6 ++---- core/src/main/scala/spark/metrics/source/Source.scala | 3 +-- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 3 ++- project/SparkBuild.scala | 2 +- 7 files changed, 11 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index f1d9d5e442..1e59a4d47d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -69,8 +69,6 @@ import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} import spark.metrics._ -import scala.util.DynamicVariable - /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 204049d927..4a1d341f5d 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -31,7 +31,6 @@ import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils import spark.api.python.PythonWorkerFactory -import spark.metrics._ /** diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index f097a631c0..6a40885b78 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -1,18 +1,18 @@ package spark.metrics.sink -import java.util.Properties - import com.codahale.metrics.{JmxReporter, MetricRegistry} +import java.util.Properties + class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() - + override def start() { reporter.start() } - + override def stop() { reporter.stop() } - + } diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 8f0870c1a0..79f505079c 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -3,15 +3,13 @@ package spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} -import java.util.{Map, HashMap => JHashMap} - class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() - + val gcMetricSet = new GarbageCollectorMetricSet val memGaugeSet = new MemoryUsageGaugeSet - + metricRegistry.registerAll(gcMetricSet) metricRegistry.registerAll(memGaugeSet) } diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index 17cbe2f85a..5607e2c40a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,9 +1,8 @@ package spark.metrics.source -import com.codahale.metrics.MetricSet import com.codahale.metrics.MetricRegistry trait Source { def sourceName: String - def metricRegistry: MetricRegistry + def metricRegistry: MetricRegistry } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 781e49bdec..9b45fc2938 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -30,7 +30,7 @@ import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialRe import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import spark.metrics.MetricsSystem + /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal @@ -126,6 +126,7 @@ class DAGScheduler( val resultStageToJob = new HashMap[Stage, ActiveJob] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) + // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 825319d3de..d4d70afdd5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -179,7 +179,7 @@ object SparkBuild extends Build { "net.liftweb" % "lift-json_2.9.2" % "2.5", "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", - "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0" ) ++ ( From ed1a3bc206c01974eedd0b1fb1deec183376b5c6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:11:10 +0800 Subject: [PATCH 144/221] continue to refactor code style and functions --- core/src/main/scala/spark/SparkContext.scala | 6 +- .../scala/spark/deploy/master/Master.scala | 6 +- .../deploy/master/MasterInstrumentation.scala | 19 ++-- .../deploy/worker/WorkerInstrumentation.scala | 35 ++++---- .../main/scala/spark/executor/Executor.scala | 8 +- .../executor/ExecutorInstrumentation.scala | 47 +++++----- .../scala/spark/metrics/MetricsConfig.scala | 88 +++++++++---------- .../scala/spark/metrics/MetricsSystem.scala | 40 ++++----- .../spark/metrics/sink/ConsoleSink.scala | 30 +++---- .../scala/spark/metrics/sink/CsvSink.scala | 40 ++++----- .../spark/scheduler/DAGSchedulerSource.scala | 23 +++-- .../spark/storage/BlockManagerSource.scala | 14 +-- .../spark/metrics/MetricsConfigSuite.scala | 32 ++++--- .../spark/metrics/MetricsSystemSuite.scala | 14 +-- 14 files changed, 195 insertions(+), 207 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1e59a4d47d..77cb0ee0cd 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -274,9 +274,9 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - def initDriverMetrics() = { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + def initDriverMetrics() { + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) } initDriverMetrics() diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index cc0b2d4295..5f67366eb6 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -57,7 +57,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") - + val masterInstrumentation = new MasterInstrumentation(this) val masterPublicAddress = { @@ -76,7 +76,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - + Master.metricsSystem.registerSource(masterInstrumentation) Master.metricsSystem.start() } @@ -322,7 +322,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } - + override def postStop() { Master.metricsSystem.stop() } diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 61a561c955..4c3708cc4c 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -5,24 +5,21 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source private[spark] class MasterInstrumentation(val master: Master) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "master" - metricRegistry.register(MetricRegistry.name("workers","number"), - new Gauge[Int] { + // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { override def getValue: Int = master.workers.size }) - + // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name("apps", "number"), - new Gauge[Int] { - override def getValue: Int = master.apps.size + metricRegistry.register(MetricRegistry.name("apps", "number"), new Gauge[Int] { + override def getValue: Int = master.apps.size }) // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), - new Gauge[Int] { - override def getValue: Int = master.waitingApps.size + metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { + override def getValue: Int = master.waitingApps.size }) - } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 94c20a98c1..c76c0b4711 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -8,32 +8,27 @@ private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name("executor", "number"), - new Gauge[Int] { - override def getValue: Int = worker.executors.size + metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { + override def getValue: Int = worker.executors.size }) - + // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name("core_used", "number"), - new Gauge[Int] { - override def getValue: Int = worker.coresUsed + metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresUsed }) - + // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), - new Gauge[Int] { - override def getValue: Int = worker.memoryUsed + metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryUsed }) - + // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name("core_free", "number"), - new Gauge[Int] { - override def getValue: Int = worker.coresFree + metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresFree }) - + // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), - new Gauge[Int] { - override def getValue: Int = worker.memoryFree - }) + metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryFree + }) } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 7179ed84a8..4ea05dec1c 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -69,7 +69,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert override def uncaughtException(thread: Thread, exception: Throwable) { try { logError("Uncaught exception in thread " + thread, exception) - + // We may have been called from a shutdown hook. If so, we must not call System.exit(). // (If we do, we will deadlock.) if (!Utils.inShutdown()) { @@ -86,14 +86,14 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } } ) - + val executorInstrumentation = new ExecutorInstrumentation(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) - SparkEnv.set(env) + SparkEnv.set(env) env.metricsSystem.registerSource(executorInstrumentation) - + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala index ebbcbee742..ad406f41b4 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -4,32 +4,27 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Executor) extends Source{ +class ExecutorInstrumentation(val executor: Executor) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "executor" - - // Gauge for executor thread pool's actively executing task counts - metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getActiveCount() - }) - - // Gauge for executor thread pool's approximate total number of tasks that have been completed - metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), - new Gauge[Long] { - override def getValue: Long = executor.threadPool.getCompletedTaskCount() - }) - - // Gauge for executor thread pool's current number of threads - metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getPoolSize() - }) - - // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool - metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getMaximumPoolSize() - }) -} \ No newline at end of file + // Gauge for executor thread pool's actively executing task counts + metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getActiveCount() + }) + + // Gauge for executor thread pool's approximate total number of tasks that have been completed + metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { + override def getValue: Long = executor.threadPool.getCompletedTaskCount() + }) + + // Gauge for executor thread pool's current number of threads + metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getPoolSize() + }) + + // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getMaximumPoolSize() + }) +} diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 7405192058..b1f6a1e596 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -3,63 +3,63 @@ package spark.metrics import java.util.Properties import java.io.{File, FileInputStream} -import scala.collection.mutable +import scala.collection.mutable.HashMap import scala.util.matching.Regex -private [spark] class MetricsConfig(val configFile: String) { +private[spark] class MetricsConfig(val configFile: String) { val properties = new Properties() - // Add default properties in case there's no properties file - MetricsConfig.setDefaultProperties(properties) - - val confFile = new File(configFile) - if (confFile.exists()) { - var fis: FileInputStream = null - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() - } - } - - val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) - if (propertyCategories.contains(MetricsConfig.DEFAULT_PREFIX)) { - import scala.collection.JavaConversions._ - val defaultProperty = propertyCategories(MetricsConfig.DEFAULT_PREFIX) - for ((inst, prop) <- propertyCategories; p <- defaultProperty - if inst != MetricsConfig.DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { - prop.setProperty(p._1, p._2) - } - } - - def getInstance(inst: String) = { - propertyCategories.get(inst) match { - case Some(s) => s - case None => propertyCategories(MetricsConfig.DEFAULT_PREFIX) - } - } -} - -private[spark] object MetricsConfig { val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r - - def setDefaultProperties(prop: Properties) { + var propertyCategories: HashMap[String, Properties] = null + + private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.jmx.enabled", "default") prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") } - - def subProperties(prop: Properties, regex: Regex) = { - val subProperties = new mutable.HashMap[String, Properties] - + + def initilize() { + //Add default properties in case there's no properties file + setDefaultProperties(properties) + + val confFile = new File(configFile) + if (confFile.exists()) { + var fis: FileInputStream = null + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } + } + + propertyCategories = subProperties(properties, INSTANCE_REGEX) + if (propertyCategories.contains(DEFAULT_PREFIX)) { + import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(DEFAULT_PREFIX) + for ((inst, prop) <- propertyCategories; p <- defaultProperty + if inst != DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { + prop.setProperty(p._1, p._2) + } + } + } + + def subProperties(prop: Properties, regex: Regex): HashMap[String, Properties] = { + val subProperties = new HashMap[String, Properties] import scala.collection.JavaConversions._ - prop.foreach { kv => + prop.foreach { kv => if (regex.findPrefixOf(kv._1) != None) { val regex(prefix, suffix) = kv._1 subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) } } - subProperties } -} \ No newline at end of file + + def getInstance(inst: String): Properties = { + propertyCategories.get(inst) match { + case Some(s) => s + case None => propertyCategories(DEFAULT_PREFIX) + } + } +} + diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index bf4487e0fc..54f6c6e4da 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -13,34 +13,37 @@ import spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - + val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") val metricsConfig = new MetricsConfig(confFile) - + val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] val registry = new MetricRegistry() - + + val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") + + metricsConfig.initilize() registerSources() registerSinks() - + def start() { sinks.foreach(_.start) } - + def stop() { sinks.foreach(_.stop) } - + def registerSource(source: Source) { sources += source registry.register(source.sourceName, source.metricRegistry) } - + def registerSources() { val instConfig = metricsConfig.getInstance(instance) - val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - + val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") @@ -52,14 +55,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } } } - + def registerSinks() { val instConfig = metricsConfig.getInstance(instance) - val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - + val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) + sinkConfigs.foreach { kv => - val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { - MetricsSystem.DEFAULT_SINKS(kv._1) + val classPath = if (DEFAULT_SINKS.contains(kv._1)) { + DEFAULT_SINKS(kv._1) } else { // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") @@ -76,17 +79,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } private[spark] object MetricsSystem { - val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") - val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val timeUnits = Map( - "millisecond" -> TimeUnit.MILLISECONDS, + "illisecond" -> TimeUnit.MILLISECONDS, "second" -> TimeUnit.SECONDS, "minute" -> TimeUnit.MINUTES, "hour" -> TimeUnit.HOURS, "day" -> TimeUnit.DAYS) - - def createMetricsSystem(instance: String) = new MetricsSystem(instance) + + def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) } diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index d7b7a9e501..c67c0ee912 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,34 +8,34 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { + + val CONSOLE_DEFAULT_PERIOD = "10" + val CONSOLE_DEFAULT_UNIT = "second" + + val CONSOLE_KEY_PERIOD = "period" + val CONSOLE_KEY_UNIT = "unit" + + val pollPeriod = Option(property.getProperty(CONSOLE_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => ConsoleSink.CONSOLE_DEFAULT_PERIOD.toInt + case None => CONSOLE_DEFAULT_PERIOD.toInt } - - val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { + + val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + case None => MetricsSystem.timeUnits(CONSOLE_DEFAULT_UNIT) } - + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() override def start() { - reporter.start(pollPeriod, pollUnit) + reporter.start(pollPeriod, pollUnit) } - + override def stop() { reporter.stop() } } -object ConsoleSink { - val CONSOLE_DEFAULT_PERIOD = "10" - val CONSOLE_DEFAULT_UNIT = "second" - - val CONSOLE_KEY_PERIOD = "period" - val CONSOLE_KEY_UNIT = "unit" -} diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index e6c5bffd3c..a8ca819e87 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -9,21 +9,29 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { + val CSV_KEY_PERIOD = "period" + val CSV_KEY_UNIT = "unit" + val CSV_KEY_DIR = "directory" + + val CSV_DEFAULT_PERIOD = "10" + val CSV_DEFAULT_UNIT = "second" + val CSV_DEFAULT_DIR = "/tmp/" + + val pollPeriod = Option(property.getProperty(CSV_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CsvSink.CSV_DEFAULT_PERIOD.toInt + case None => CSV_DEFAULT_PERIOD.toInt } - - val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { + + val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + case None => MetricsSystem.timeUnits(CSV_DEFAULT_UNIT) } - - val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { + + val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { case Some(s) => s - case None => CsvSink.CSV_DEFAULT_DIR + case None => CSV_DEFAULT_DIR } - + val reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) @@ -31,21 +39,11 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si .build(new File(pollDir)) override def start() { - reporter.start(pollPeriod, pollUnit) + reporter.start(pollPeriod, pollUnit) } - + override def stop() { reporter.stop() } } -object CsvSink { - val CSV_KEY_PERIOD = "period" - val CSV_KEY_UNIT = "unit" - val CSV_KEY_DIR = "directory" - - val CSV_DEFAULT_PERIOD = "10" - val CSV_DEFAULT_UNIT = "second" - val CSV_DEFAULT_DIR = "/tmp/" -} - diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala index 57aa74512c..38158b8a2b 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -5,27 +5,26 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "DAGScheduler" - - metricRegistry.register(MetricRegistry.name("stage","failedStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.failed.size + metricRegistry.register(MetricRegistry.name("stage", "failedStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.failed.size }) - metricRegistry.register(MetricRegistry.name("stage","runningStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.running.size + metricRegistry.register(MetricRegistry.name("stage", "runningStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.running.size }) - metricRegistry.register(MetricRegistry.name("stage","waitingStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.waiting.size + metricRegistry.register(MetricRegistry.name("stage", "waitingStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.waiting.size }) - metricRegistry.register(MetricRegistry.name("job","allJobs"), new Gauge[Int] { - override def getValue: Int = dagScheduler.nextRunId.get() + metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.nextRunId.get() }) - metricRegistry.register(MetricRegistry.name("job","ActiveJobs"), new Gauge[Int] { - override def getValue: Int = dagScheduler.activeJobs.size + metricRegistry.register(MetricRegistry.name("job", "ActiveJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.activeJobs.size }) } diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index c0ce9259c8..f964827102 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -6,29 +6,29 @@ import spark.metrics.source.Source import spark.storage._ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory","maxMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "maxMem"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) maxMem } }) - metricRegistry.register(MetricRegistry.name("memory","remainingMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "remainingMem"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) remainingMem } }) - metricRegistry.register(MetricRegistry.name("disk","diskSpaceUsed"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) diskSpaceUsed } }) diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index f4c83cb644..bb1be4f4fc 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -9,60 +9,64 @@ import spark.metrics._ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + before { filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile() } test("MetricsConfig with default properties") { val conf = new MetricsConfig("dummy-file") + conf.initilize() + assert(conf.properties.size() === 2) assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") assert(conf.properties.getProperty("test-for-dummy") === null) - + val property = conf.getInstance("random") assert(property.size() === 2) assert(property.getProperty("sink.jmx.enabled") === "default") assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") } - + test("MetricsConfig with properties set") { val conf = new MetricsConfig(filePath) - + conf.initilize() + val masterProp = conf.getInstance("master") assert(masterProp.size() === 4) assert(masterProp.getProperty("sink.console.period") === "20") assert(masterProp.getProperty("sink.console.unit") === "minute") assert(masterProp.getProperty("sink.jmx.enabled") === "default") assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") - + val workerProp = conf.getInstance("worker") assert(workerProp.size() === 4) assert(workerProp.getProperty("sink.console.period") === "10") assert(workerProp.getProperty("sink.console.unit") === "second") } - + test("MetricsConfig with subProperties") { val conf = new MetricsConfig(filePath) - + conf.initilize() + val propCategories = conf.propertyCategories assert(propCategories.size === 2) - + val masterProp = conf.getInstance("master") - val sourceProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) + val sourceProps = conf.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) assert(sourceProps.size === 1) assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") - - val sinkProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SINK_REGEX) + + val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX) assert(sinkProps.size === 2) assert(sinkProps.contains("console")) assert(sinkProps.contains("jmx")) - + val consoleProps = sinkProps("console") assert(consoleProps.size() === 2) - + val jmxProps = sinkProps("jmx") assert(jmxProps.size() === 1) - } + } } diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 967be6ec47..f29bb9db67 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -9,32 +9,32 @@ import spark.metrics._ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf.file", filePath) } - + test("MetricsSystem with default config") { val metricsSystem = MetricsSystem.createMetricsSystem("default") val sources = metricsSystem.sources val sinks = metricsSystem.sinks - + assert(sources.length === 1) assert(sinks.length === 1) assert(sources(0).sourceName === "jvm") } - + test("MetricsSystem with sources add") { val metricsSystem = MetricsSystem.createMetricsSystem("test") val sources = metricsSystem.sources val sinks = metricsSystem.sinks - + assert(sources.length === 1) assert(sinks.length === 2) - + val source = new spark.deploy.master.MasterInstrumentation(null) metricsSystem.registerSource(source) assert(sources.length === 2) - } + } } From 5b4a2f2094258920613f50c0d337e1e92c08d012 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:31:53 +0800 Subject: [PATCH 145/221] Add metrics config template file --- conf/metrics.properties | 15 --------------- conf/metrics.properties.template | 14 ++++++++++++++ 2 files changed, 14 insertions(+), 15 deletions(-) delete mode 100644 conf/metrics.properties create mode 100644 conf/metrics.properties.template diff --git a/conf/metrics.properties b/conf/metrics.properties deleted file mode 100644 index 2ea3aaf345..0000000000 --- a/conf/metrics.properties +++ /dev/null @@ -1,15 +0,0 @@ -# syntax: [instance].[sink|source].[name].[options] - -*.sink.console.period=10 - -*.sink.console.unit=second - -*.sink.console.class=spark.metrics.sink.ConsoleSink - -master.sink.console.period=10 - -master.sink.console.unit=second - -worker.sink.console.period=20 - -worker.sink.console.unit=second diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template new file mode 100644 index 0000000000..b4204e1deb --- /dev/null +++ b/conf/metrics.properties.template @@ -0,0 +1,14 @@ +# syntax: [instance].[sink|source].[name].[options] + +#*.sink.console.period=10 + +#*.sink.console.unit=second + +#master.source.jvm.class=spark.metrics.source.JvmSource + +#worker.source.jvm.class=spark.metrics.source.JvmSource + +#driver.source.jvm.class=spark.metrics.source.JvmSource + +#executor.source.jvm.class=spark.metrics.source.JvmSource + From 05637de8423da85c5934cdfa8f07254133a58474 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 21:16:39 +0800 Subject: [PATCH 146/221] Change class xxxInstrumentation to class xxxSource --- core/src/main/scala/spark/deploy/master/Master.scala | 4 ++-- .../{MasterInstrumentation.scala => MasterSource.scala} | 2 +- core/src/main/scala/spark/deploy/worker/Worker.scala | 4 ++-- .../{WorkerInstrumentation.scala => WorkerSource.scala} | 2 +- core/src/main/scala/spark/executor/Executor.scala | 4 ++-- .../{ExecutorInstrumentation.scala => ExecutorSource.scala} | 2 +- core/src/test/scala/spark/metrics/MetricsSystemSuite.scala | 2 +- 7 files changed, 10 insertions(+), 10 deletions(-) rename core/src/main/scala/spark/deploy/master/{MasterInstrumentation.scala => MasterSource.scala} (90%) rename core/src/main/scala/spark/deploy/worker/{WorkerInstrumentation.scala => WorkerSource.scala} (93%) rename core/src/main/scala/spark/executor/{ExecutorInstrumentation.scala => ExecutorSource.scala} (94%) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 5f67366eb6..1d592206c0 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -58,7 +58,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val masterInstrumentation = new MasterInstrumentation(this) + val masterSource = new MasterSource(this) val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -77,7 +77,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - Master.metricsSystem.registerSource(masterInstrumentation) + Master.metricsSystem.registerSource(masterSource) Master.metricsSystem.start() } diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala similarity index 90% rename from core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala rename to core/src/main/scala/spark/deploy/master/MasterSource.scala index 4c3708cc4c..f94e5b2c34 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -private[spark] class MasterInstrumentation(val master: Master) extends Source { +private[spark] class MasterSource(val master: Master) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "master" diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index eaa1c1806f..5c0f77fd75 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -68,7 +68,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 - val workerInstrumentation = new WorkerInstrumentation(this) + val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -102,7 +102,7 @@ private[spark] class Worker( connectToMaster() startWebUi() - Worker.metricsSystem.registerSource(workerInstrumentation) + Worker.metricsSystem.registerSource(workerSource) Worker.metricsSystem.start() } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala similarity index 93% rename from core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala rename to core/src/main/scala/spark/deploy/worker/WorkerSource.scala index c76c0b4711..539eac71bd 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { +private[spark] class WorkerSource(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 4ea05dec1c..8a74a8d853 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -87,12 +87,12 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } ) - val executorInstrumentation = new ExecutorInstrumentation(this) + val executorSource = new ExecutorSource(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) SparkEnv.set(env) - env.metricsSystem.registerSource(executorInstrumentation) + env.metricsSystem.registerSource(executorSource) private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorSource.scala similarity index 94% rename from core/src/main/scala/spark/executor/ExecutorInstrumentation.scala rename to core/src/main/scala/spark/executor/ExecutorSource.scala index ad406f41b4..d8b531cb58 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Executor) extends Source { +class ExecutorSource(val executor: Executor) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "executor" diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index f29bb9db67..462c28e894 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -33,7 +33,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { assert(sources.length === 1) assert(sinks.length === 2) - val source = new spark.deploy.master.MasterInstrumentation(null) + val source = new spark.deploy.master.MasterSource(null) metricsSystem.registerSource(source) assert(sources.length === 2) } From 8d1ef7f2dfc453137b8dbbb72a7f1ca93b57bb15 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 4 Jul 2013 10:11:30 +0800 Subject: [PATCH 147/221] Code style changes --- conf/metrics.properties.template | 2 ++ .../src/main/scala/spark/deploy/master/MasterSource.scala | 4 ++-- core/src/main/scala/spark/metrics/MetricsConfig.scala | 8 ++++---- core/src/main/scala/spark/metrics/sink/ConsoleSink.scala | 1 - 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index b4204e1deb..c7e24aa36c 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,5 +1,7 @@ # syntax: [instance].[sink|source].[name].[options] +#*.sink.console.class=spark.metrics.sink.ConsoleSink + #*.sink.console.period=10 #*.sink.console.unit=second diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala index f94e5b2c34..65c22320d6 100644 --- a/core/src/main/scala/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -1,6 +1,6 @@ package spark.deploy.master -import com.codahale.metrics.{Gauge,MetricRegistry} +import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source @@ -10,7 +10,7 @@ private[spark] class MasterSource(val master: Master) extends Source { // Gauge for worker numbers in cluster metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { - override def getValue: Int = master.workers.size + override def getValue: Int = master.workers.size }) // Gauge for application numbers in cluster diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index b1f6a1e596..5066b7ac22 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -3,14 +3,14 @@ package spark.metrics import java.util.Properties import java.io.{File, FileInputStream} -import scala.collection.mutable.HashMap +import scala.collection.mutable import scala.util.matching.Regex private[spark] class MetricsConfig(val configFile: String) { val properties = new Properties() val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r - var propertyCategories: HashMap[String, Properties] = null + var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.jmx.enabled", "default") @@ -43,8 +43,8 @@ private[spark] class MetricsConfig(val configFile: String) { } } - def subProperties(prop: Properties, regex: Regex): HashMap[String, Properties] = { - val subProperties = new HashMap[String, Properties] + def subProperties(prop: Properties, regex: Regex): mutable.HashMap[String, Properties] = { + val subProperties = new mutable.HashMap[String, Properties] import scala.collection.JavaConversions._ prop.foreach { kv => if (regex.findPrefixOf(kv._1) != None) { diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index c67c0ee912..437f24a575 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,7 +8,6 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val CONSOLE_DEFAULT_PERIOD = "10" val CONSOLE_DEFAULT_UNIT = "second" From 31ec72b243459e0d1d9c89033328e1d79f3aa76f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 16 Jul 2013 10:57:19 +0800 Subject: [PATCH 148/221] Code refactor according to comments --- conf/metrics.properties.template | 73 ++++++++++++++++- .../scala/spark/deploy/master/Master.scala | 11 +-- .../spark/deploy/master/MasterSource.scala | 2 +- .../scala/spark/deploy/worker/Worker.scala | 11 +-- .../spark/deploy/worker/WorkerSource.scala | 12 +-- .../scala/spark/executor/ExecutorSource.scala | 8 +- .../scala/spark/metrics/MetricsConfig.scala | 50 +++++++----- .../scala/spark/metrics/MetricsSystem.scala | 81 ++++++++++++++----- .../spark/metrics/sink/ConsoleSink.scala | 12 +-- .../scala/spark/metrics/sink/CsvSink.scala | 12 +-- .../spark/scheduler/DAGSchedulerSource.scala | 10 +-- .../spark/storage/BlockManagerSource.scala | 31 ++++--- .../resources/test_metrics_config.properties | 6 +- .../resources/test_metrics_system.properties | 4 +- .../spark/metrics/MetricsConfigSuite.scala | 38 ++++----- .../spark/metrics/MetricsSystemSuite.scala | 13 ++- 16 files changed, 249 insertions(+), 125 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index c7e24aa36c..0486ca4c79 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,11 +1,82 @@ # syntax: [instance].[sink|source].[name].[options] +# "instance" specify "who" (the role) use metrics system. In spark there are +# several roles like master, worker, executor, driver, these roles will +# create metrics system for monitoring. So instance represents these roles. +# Currently in Spark, several instances have already implemented: master, +# worker, executor, driver. +# +# [instance] field can be "master", "worker", "executor", "driver", which means +# only the specified instance has this property. +# a wild card "*" can be used to represent instance name, which means all the +# instances will have this property. +# +# "source" specify "where" (source) to collect metrics data. In metrics system, +# there exists two kinds of source: +# 1. Spark internal source, like MasterSource, WorkerSource, etc, which will +# collect Spark component's internal state, these sources are related to +# instance and will be added after specific metrics system is created. +# 2. Common source, like JvmSource, which will collect low level state, is +# configured by configuration and loaded through reflection. +# +# "sink" specify "where" (destination) to output metrics data to. Several sinks +# can be coexisted and flush metrics to all these sinks. +# +# [sink|source] field specify this property is source related or sink, this +# field can only be source or sink. +# +# [name] field specify the name of source or sink, this is custom defined. +# +# [options] field is the specific property of this source or sink, this source +# or sink is responsible for parsing this property. +# +# Notes: +# 1. Sinks should be added through configuration, like console sink, class +# full name should be specified by class property. +# 2. Some sinks can specify polling period, like console sink, which is 10 seconds, +# it should be attention minimal polling period is 1 seconds, any period +# below than 1s is illegal. +# 3. Wild card property can be overlapped by specific instance property, for +# example, *.sink.console.period can be overlapped by master.sink.console.period. +# 4. A metrics specific configuration +# "spark.metrics.conf=${SPARK_HOME}/conf/metrics.properties" should be +# added to Java property using -Dspark.metrics.conf=xxx if you want to +# customize metrics system, or you can put it in ${SPARK_HOME}/conf, +# metrics system will search and load it automatically. + +# Enable JmxSink for all instances by class name +#*.sink.jmx.class=spark.metrics.sink.JmxSink + +# Enable ConsoleSink for all instances by class name #*.sink.console.class=spark.metrics.sink.ConsoleSink +# Polling period for ConsoleSink #*.sink.console.period=10 -#*.sink.console.unit=second +#*.sink.console.unit=seconds +# Master instance overlap polling period +#master.sink.console.period=15 + +#master.sink.console.unit=seconds + +# Enable CsvSink for all instances +#*.sink.csv.class=spark.metrics.sink.CsvSink + +# Polling period for CsvSink +#*.sink.csv.period=1 + +#*.sink.csv.unit=minutes + +# Polling directory for CsvSink +#*.sink.csv.directory=/tmp/ + +# Worker instance overlap polling period +#worker.sink.csv.period=10 + +#worker.sink.csv.unit=minutes + +# Enable jvm source for instance master, worker, driver and executor #master.source.jvm.class=spark.metrics.source.JvmSource #worker.source.jvm.class=spark.metrics.source.JvmSource diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 1d592206c0..9692af5295 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -58,6 +58,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") + val metricsSystem = MetricsSystem.createMetricsSystem("master") val masterSource = new MasterSource(this) val masterPublicAddress = { @@ -77,12 +78,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - Master.metricsSystem.registerSource(masterSource) - Master.metricsSystem.start() + metricsSystem.registerSource(masterSource) + metricsSystem.start() } override def postStop() { webUi.stop() + metricsSystem.stop() } override def receive = { @@ -322,17 +324,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } - - override def postStop() { - Master.metricsSystem.stop() - } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r - private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala index 65c22320d6..b8cfa6a773 100644 --- a/core/src/main/scala/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -19,7 +19,7 @@ private[spark] class MasterSource(val master: Master) extends Source { }) // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("waitingApps", "number"), new Gauge[Int] { override def getValue: Int = master.waitingApps.size }) } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 5c0f77fd75..8fa0d12b82 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -68,6 +68,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + val metricsSystem = MetricsSystem.createMetricsSystem("worker") val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed @@ -100,10 +101,9 @@ private[spark] class Worker( webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() - startWebUi() - Worker.metricsSystem.registerSource(workerSource) - Worker.metricsSystem.start() + metricsSystem.registerSource(workerSource) + metricsSystem.start() } def connectToMaster() { @@ -185,14 +185,11 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) webUi.stop() - - Worker.metricsSystem.stop() + metricsSystem.stop() } } private[spark] object Worker { - private val metricsSystem = MetricsSystem.createMetricsSystem("worker") - def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala index 539eac71bd..39cb8e5690 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala @@ -8,27 +8,27 @@ private[spark] class WorkerSource(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("executors", "number"), new Gauge[Int] { override def getValue: Int = worker.executors.size }) // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("coresUsed", "number"), new Gauge[Int] { override def getValue: Int = worker.coresUsed }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("memUsed", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryUsed }) // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("coresFree", "number"), new Gauge[Int] { override def getValue: Int = worker.coresFree }) - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { + // Gauge for memory free of this worker + metricRegistry.register(MetricRegistry.name("memFree", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryFree }) } diff --git a/core/src/main/scala/spark/executor/ExecutorSource.scala b/core/src/main/scala/spark/executor/ExecutorSource.scala index d8b531cb58..94116edfcf 100644 --- a/core/src/main/scala/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/spark/executor/ExecutorSource.scala @@ -9,22 +9,22 @@ class ExecutorSource(val executor: Executor) extends Source { val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts - metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "activeTask", "count"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed - metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("threadpool", "completeTask", "count"), new Gauge[Long] { override def getValue: Long = executor.threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads - metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "currentPool", "size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool - metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "maxPool", "size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) } diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 5066b7ac22..ed505b0aa7 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -1,44 +1,58 @@ package spark.metrics import java.util.Properties -import java.io.{File, FileInputStream} +import java.io.{File, FileInputStream, InputStream, IOException} import scala.collection.mutable import scala.util.matching.Regex -private[spark] class MetricsConfig(val configFile: String) { - val properties = new Properties() +import spark.Logging + +private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { + initLogging() + val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + val METRICS_CONF = "metrics.properties" + + val properties = new Properties() var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { - prop.setProperty("*.sink.jmx.enabled", "default") - prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + // empty function, any default property can be set here } - def initilize() { + def initialize() { //Add default properties in case there's no properties file setDefaultProperties(properties) - val confFile = new File(configFile) - if (confFile.exists()) { - var fis: FileInputStream = null - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() + // If spark.metrics.conf is not set, try to get file in class path + var is: InputStream = null + try { + is = configFile match { + case Some(f) => new FileInputStream(f) + case None => getClass.getClassLoader.getResourceAsStream(METRICS_CONF) } + + if (is != null) { + properties.load(is) + } + } catch { + case e: Exception => logError("Error loading configure file", e) + } finally { + if (is != null) is.close() } propertyCategories = subProperties(properties, INSTANCE_REGEX) if (propertyCategories.contains(DEFAULT_PREFIX)) { import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(DEFAULT_PREFIX) - for ((inst, prop) <- propertyCategories; p <- defaultProperty - if inst != DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { - prop.setProperty(p._1, p._2) + for { (inst, prop) <- propertyCategories + if (inst != DEFAULT_PREFIX) + (k, v) <- defaultProperty + if (prop.getProperty(k) == null) } { + prop.setProperty(k, v) } } } @@ -58,7 +72,7 @@ private[spark] class MetricsConfig(val configFile: String) { def getInstance(inst: String): Properties = { propertyCategories.get(inst) match { case Some(s) => s - case None => propertyCategories(DEFAULT_PREFIX) + case None => propertyCategories.getOrElse(DEFAULT_PREFIX, new Properties) } } } diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 54f6c6e4da..2f87577ff3 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -11,19 +11,51 @@ import spark.Logging import spark.metrics.sink.Sink import spark.metrics.source.Source +/** + * Spark Metrics System, created by specific "instance", combined by source, + * sink, periodically poll source metrics data to sink destinations. + * + * "instance" specify "who" (the role) use metrics system. In spark there are several roles + * like master, worker, executor, client driver, these roles will create metrics system + * for monitoring. So instance represents these roles. Currently in Spark, several instances + * have already implemented: master, worker, executor, driver. + * + * "source" specify "where" (source) to collect metrics data. In metrics system, there exists + * two kinds of source: + * 1. Spark internal source, like MasterSource, WorkerSource, etc, which will collect + * Spark component's internal state, these sources are related to instance and will be + * added after specific metrics system is created. + * 2. Common source, like JvmSource, which will collect low level state, is configured by + * configuration and loaded through reflection. + * + * "sink" specify "where" (destination) to output metrics data to. Several sinks can be + * coexisted and flush metrics to all these sinks. + * + * Metrics configuration format is like below: + * [instance].[sink|source].[name].[options] = xxxx + * + * [instance] can be "master", "worker", "executor", "driver", which means only the specified + * instance has this property. + * wild card "*" can be used to replace instance name, which means all the instances will have + * this property. + * + * [sink|source] means this property belongs to source or sink. This field can only be source or sink. + * + * [name] specify the name of sink or source, it is custom defined. + * + * [options] is the specific property of this source or sink. + */ private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") - val metricsConfig = new MetricsConfig(confFile) + val confFile = System.getProperty("spark.metrics.conf") + val metricsConfig = new MetricsConfig(Option(confFile)) val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] val registry = new MetricRegistry() - val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") - - metricsConfig.initilize() + metricsConfig.initialize() registerSources() registerSinks() @@ -37,7 +69,11 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - registry.register(source.sourceName, source.metricRegistry) + try { + registry.register(source.sourceName, source.metricRegistry) + } catch { + case e: IllegalArgumentException => logInfo("Metrics already registered", e) + } } def registerSources() { @@ -51,7 +87,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val source = Class.forName(classPath).newInstance() registerSource(source.asInstanceOf[Source]) } catch { - case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Source class " + classPath + " cannot be instantialized", e) } } } @@ -61,18 +97,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) sinkConfigs.foreach { kv => - val classPath = if (DEFAULT_SINKS.contains(kv._1)) { - DEFAULT_SINKS(kv._1) - } else { - // For non-default sink, a property class should be set and create using reflection - kv._2.getProperty("class") - } + val classPath = kv._2.getProperty("class") try { - val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) + val sink = Class.forName(classPath) + .getConstructor(classOf[Properties], classOf[MetricRegistry]) .newInstance(kv._2, registry) sinks += sink.asInstanceOf[Sink] } catch { - case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } } } @@ -81,12 +113,17 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin private[spark] object MetricsSystem { val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val timeUnits = Map( - "illisecond" -> TimeUnit.MILLISECONDS, - "second" -> TimeUnit.SECONDS, - "minute" -> TimeUnit.MINUTES, - "hour" -> TimeUnit.HOURS, - "day" -> TimeUnit.DAYS) - def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) + val MINIMAL_POLL_UNIT = TimeUnit.SECONDS + val MINIMAL_POLL_PERIOD = 1 + + def checkMinimalPollingPeriod(pollUnit: TimeUnit, pollPeriod: Int) { + val period = MINIMAL_POLL_UNIT.convert(pollPeriod, pollUnit) + if (period < MINIMAL_POLL_PERIOD) { + throw new IllegalArgumentException("Polling period " + pollPeriod + " " + pollUnit + + " below than minimal polling period ") + } + } + + def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) } diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 437f24a575..eaaac5d153 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,22 +8,24 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val CONSOLE_DEFAULT_PERIOD = "10" - val CONSOLE_DEFAULT_UNIT = "second" + val CONSOLE_DEFAULT_PERIOD = 10 + val CONSOLE_DEFAULT_UNIT = "SECONDS" val CONSOLE_KEY_PERIOD = "period" val CONSOLE_KEY_UNIT = "unit" val pollPeriod = Option(property.getProperty(CONSOLE_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CONSOLE_DEFAULT_PERIOD.toInt + case None => CONSOLE_DEFAULT_PERIOD } val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { - case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CONSOLE_DEFAULT_UNIT) + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(CONSOLE_DEFAULT_UNIT) } + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index a8ca819e87..aa5bff0d34 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -13,19 +13,21 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si val CSV_KEY_UNIT = "unit" val CSV_KEY_DIR = "directory" - val CSV_DEFAULT_PERIOD = "10" - val CSV_DEFAULT_UNIT = "second" + val CSV_DEFAULT_PERIOD = 10 + val CSV_DEFAULT_UNIT = "SECONDS" val CSV_DEFAULT_DIR = "/tmp/" val pollPeriod = Option(property.getProperty(CSV_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CSV_DEFAULT_PERIOD.toInt + case None => CSV_DEFAULT_PERIOD } val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { - case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CSV_DEFAULT_UNIT) + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { case Some(s) => s diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala index 38158b8a2b..87d27cc70d 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -8,23 +8,23 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends val metricRegistry = new MetricRegistry() val sourceName = "DAGScheduler" - metricRegistry.register(MetricRegistry.name("stage", "failedStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "failedStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.failed.size }) - metricRegistry.register(MetricRegistry.name("stage", "runningStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "runningStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.running.size }) - metricRegistry.register(MetricRegistry.name("stage", "waitingStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "waitingStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.waiting.size }) - metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("job", "allJobs", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.nextRunId.get() }) - metricRegistry.register(MetricRegistry.name("job", "ActiveJobs"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("job", "activeJobs", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.activeJobs.size }) } diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index f964827102..4faa715c94 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -9,27 +9,40 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends val metricRegistry = new MetricRegistry() val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory", "maxMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "maxMem", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - maxMem + val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + maxMem / 1024 / 1024 } }) - metricRegistry.register(MetricRegistry.name("memory", "remainingMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "remainingMem", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - remainingMem + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + remainingMem / 1024 / 1024 } }) - metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "memUsed", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) - diskSpaceUsed + val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + (maxMem - remainingMem) / 1024 / 1024 + } + }) + + metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed", "MBytes"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val diskSpaceUsed = storageStatusList + .flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_ + _) + .getOrElse(0L) + + diskSpaceUsed / 1024 / 1024 } }) } diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties index 2011940003..2b31ddf2eb 100644 --- a/core/src/test/resources/test_metrics_config.properties +++ b/core/src/test/resources/test_metrics_config.properties @@ -1,6 +1,6 @@ *.sink.console.period = 10 -*.sink.console.unit = second +*.sink.console.unit = seconds *.source.jvm.class = spark.metrics.source.JvmSource master.sink.console.period = 20 -master.sink.console.unit = minute - +master.sink.console.unit = minutes + diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties index 06afbc6625..d5479f0298 100644 --- a/core/src/test/resources/test_metrics_system.properties +++ b/core/src/test/resources/test_metrics_system.properties @@ -1,7 +1,7 @@ *.sink.console.period = 10 -*.sink.console.unit = second +*.sink.console.unit = seconds test.sink.console.class = spark.metrics.sink.ConsoleSink test.sink.dummy.class = spark.metrics.sink.DummySink test.source.dummy.class = spark.metrics.source.DummySource test.sink.console.period = 20 -test.sink.console.unit = minute +test.sink.console.unit = minutes diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index bb1be4f4fc..87cd2ffad2 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -15,40 +15,36 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { } test("MetricsConfig with default properties") { - val conf = new MetricsConfig("dummy-file") - conf.initilize() + val conf = new MetricsConfig(Option("dummy-file")) + conf.initialize() - assert(conf.properties.size() === 2) - assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") - assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") + assert(conf.properties.size() === 0) assert(conf.properties.getProperty("test-for-dummy") === null) val property = conf.getInstance("random") - assert(property.size() === 2) - assert(property.getProperty("sink.jmx.enabled") === "default") - assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") + assert(property.size() === 0) } test("MetricsConfig with properties set") { - val conf = new MetricsConfig(filePath) - conf.initilize() + val conf = new MetricsConfig(Option(filePath)) + conf.initialize() val masterProp = conf.getInstance("master") - assert(masterProp.size() === 4) + assert(masterProp.size() === 3) assert(masterProp.getProperty("sink.console.period") === "20") - assert(masterProp.getProperty("sink.console.unit") === "minute") - assert(masterProp.getProperty("sink.jmx.enabled") === "default") - assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") + assert(masterProp.getProperty("sink.console.unit") === "minutes") + assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") val workerProp = conf.getInstance("worker") - assert(workerProp.size() === 4) + assert(workerProp.size() === 3) assert(workerProp.getProperty("sink.console.period") === "10") - assert(workerProp.getProperty("sink.console.unit") === "second") + assert(workerProp.getProperty("sink.console.unit") === "seconds") + assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") } test("MetricsConfig with subProperties") { - val conf = new MetricsConfig(filePath) - conf.initilize() + val conf = new MetricsConfig(Option(filePath)) + conf.initialize() val propCategories = conf.propertyCategories assert(propCategories.size === 2) @@ -59,14 +55,10 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX) - assert(sinkProps.size === 2) + assert(sinkProps.size === 1) assert(sinkProps.contains("console")) - assert(sinkProps.contains("jmx")) val consoleProps = sinkProps("console") assert(consoleProps.size() === 2) - - val jmxProps = sinkProps("jmx") - assert(jmxProps.size() === 1) } } diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 462c28e894..c189996417 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -12,7 +12,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() - System.setProperty("spark.metrics.conf.file", filePath) + System.setProperty("spark.metrics.conf", filePath) } test("MetricsSystem with default config") { @@ -20,9 +20,8 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val sources = metricsSystem.sources val sinks = metricsSystem.sinks - assert(sources.length === 1) - assert(sinks.length === 1) - assert(sources(0).sourceName === "jvm") + assert(sources.length === 0) + assert(sinks.length === 0) } test("MetricsSystem with sources add") { @@ -30,11 +29,11 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val sources = metricsSystem.sources val sinks = metricsSystem.sinks - assert(sources.length === 1) - assert(sinks.length === 2) + assert(sources.length === 0) + assert(sinks.length === 1) val source = new spark.deploy.master.MasterSource(null) metricsSystem.registerSource(source) - assert(sources.length === 2) + assert(sources.length === 1) } } From 93c6015f82d4d27a6f09686b1e849be1cbbd0615 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 10:53:02 -0700 Subject: [PATCH 149/221] Shows task status and running tasks on Stage Page: fixes SPARK-804 and 811 --- .../spark/scheduler/cluster/TaskInfo.scala | 12 ++++++++++++ .../scala/spark/ui/exec/ExecutorsUI.scala | 5 +++++ .../scala/spark/ui/jobs/JobProgressUI.scala | 16 ++++++++++++++++ .../main/scala/spark/ui/jobs/StagePage.scala | 19 +++++++++++++------ 4 files changed, 46 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index a1ebd48b01..f840100eca 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -51,6 +51,18 @@ class TaskInfo( def running: Boolean = !finished + def status: String = { + if (running) + return "RUNNING" + if (failed) + return "FAILED" + if (successful) + return "SUCCESSFUL" + if (finished) + return "FINISHED" + "UNKNOWN" + } + def duration: Long = { if (!finished) { throw new UnsupportedOperationException("duration() called on unfinished tasks") diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index bad5c442ab..db1c902955 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -124,6 +124,10 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { if (!executorToTasksActive.contains(eid)) executorToTasksActive(eid) = HashSet[Long]() executorToTasksActive(eid) += taskStart.taskInfo.taskId + val taskList = executorToTaskInfos.getOrElse( + eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskStart.taskInfo, None, None)) + executorToTaskInfos(eid) = taskList } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { @@ -142,6 +146,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList -= ((taskEnd.taskInfo, None, None)) taskList += ((taskEnd.taskInfo, metrics, failureInfo)) executorToTaskInfos(eid) = taskList } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 44dcf82d11..6e332415db 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -65,6 +65,7 @@ private[spark] class JobProgressListener extends SparkListener { val completedStages = ListBuffer[Stage]() val failedStages = ListBuffer[Stage]() + val stageToTasksActive = HashMap[Int, HashSet[Long]]() val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = @@ -93,8 +94,22 @@ private[spark] class JobProgressListener extends SparkListener { override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = activeStages += stageSubmitted.stage + override def onTaskStart(taskStart: SparkListenerTaskStart) { + val sid = taskStart.task.stageId + if (!stageToTasksActive.contains(sid)) + stageToTasksActive(sid) = HashSet[Long]() + stageToTasksActive(sid) += taskStart.taskInfo.taskId + val taskList = stageToTaskInfos.getOrElse( + sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskStart.taskInfo, None, None)) + stageToTaskInfos(sid) = taskList + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId + if (!stageToTasksActive.contains(sid)) + stageToTasksActive(sid) = HashSet[Long]() + stageToTasksActive(sid) -= taskEnd.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -106,6 +121,7 @@ private[spark] class JobProgressListener extends SparkListener { } val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList -= ((taskEnd.taskInfo, None, None)) taskList += ((taskEnd.taskInfo, metrics, failureInfo)) stageToTaskInfos(sid) = taskList } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 292966f23a..b77b29045e 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -41,8 +41,8 @@ private[spark] class StagePage(parent: JobProgressUI) { if (!listener.stageToTaskInfos.contains(stageId)) { val content =
        -

        Summary Metrics

        No tasks have finished yet -

        Tasks

        No tasks have finished yet +

        Summary Metrics

        No tasks have started yet +

        Tasks

        No tasks have started yet
        return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } @@ -53,7 +53,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleWrite = listener.hasShuffleWrite(stageId) val taskHeaders: Seq[String] = - Seq("Task ID", "Duration", "Locality Level", "Worker", "Launch Time") ++ + Seq("Task ID", "Status", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ {if (shuffleWrite) Seq("Shuffle Write") else Nil} ++ Seq("Details") @@ -61,7 +61,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskTable = listingTable(taskHeaders, taskRow, tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => Option(t._2).isDefined) + val validTasks = tasks.filter(t => t._1.status == "SUCCESSFUL" && (Option(t._2).isDefined)) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { @@ -108,10 +108,17 @@ private[spark] class StagePage(parent: JobProgressUI) { def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = trace.map(e => {e.toString}) val (info, metrics, exception) = taskData + + val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) + else metrics.map(m => m.executorRunTime).getOrElse(1) + val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) + else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + {info.taskId} - m.executorRunTime.toString}.getOrElse("1")}> - {metrics.map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} + {info.status} + + {formatDuration} {info.taskLocality} {info.hostPort} From bd3931c87415679fef8b91390d50188c3da98430 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 11:27:17 -0700 Subject: [PATCH 150/221] Changed ifs with returns to if/else --- .../spark/scheduler/cluster/TaskInfo.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index f840100eca..afcfe96a09 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -53,14 +53,15 @@ class TaskInfo( def status: String = { if (running) - return "RUNNING" - if (failed) - return "FAILED" - if (successful) - return "SUCCESSFUL" - if (finished) - return "FINISHED" - "UNKNOWN" + "RUNNING" + else if (failed) + "FAILED" + else if (successful) + "SUCCESSFUL" + else if (finished) + "FINISHED" + else + "UNKNOWN" } def duration: Long = { From 4280e1768d09e43b55f26556e14a8eb70d9b9fad Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 12:48:48 -0700 Subject: [PATCH 151/221] Removed finished status for task info, changed name of success case --- core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index afcfe96a09..c693b722ac 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -57,9 +57,7 @@ class TaskInfo( else if (failed) "FAILED" else if (successful) - "SUCCESSFUL" - else if (finished) - "FINISHED" + "SUCCESS" else "UNKNOWN" } From 57009eef9099aa3b35b9262d9637af3dc94266a1 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 13:43:09 -0700 Subject: [PATCH 152/221] Fixed consistency of "success" status string --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index b77b29045e..654f347723 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -61,7 +61,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskTable = listingTable(taskHeaders, taskRow, tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => t._1.status == "SUCCESSFUL" && (Option(t._2).isDefined)) + val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && (Option(t._2).isDefined)) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { From 1d101928069c31aec039bb8b69c9bf4c204eead4 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 22 Jul 2013 13:11:35 -0700 Subject: [PATCH 153/221] Fix setting of SPARK_EXAMPLES_JAR --- bin/compute-classpath.sh | 11 ----------- run | 13 +++++++++++++ 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index eb270a5428..e4ce1ca848 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -76,17 +76,6 @@ function dev_classpath { CLASSPATH="$CLASSPATH:$jar" done - # 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. - if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then - # Use the JAR from the SBT build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` - fi - if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then - # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` - fi - # Add Scala standard library if [ -z "$SCALA_LIBRARY_PATH" ]; then if [ -z "$SCALA_HOME" ]; then diff --git a/run b/run index f49aa92796..4cffda4708 100755 --- a/run +++ b/run @@ -17,6 +17,8 @@ # limitations under the License. # +SCALA_VERSION=2.9.3 + # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" @@ -136,6 +138,17 @@ if [ ! -f "$FWDIR/RELEASE" ]; then echo "You need to compile Spark repl module before running this program" >&2 exit 1 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. + if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` + fi + if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the Maven build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + fi fi # Compute classpath using external script From eef678703eed96544224209b1555618968b2eb3f Mon Sep 17 00:00:00 2001 From: Xinghao Date: Wed, 24 Jul 2013 15:32:50 -0700 Subject: [PATCH 154/221] Adding SVM and Lasso, moving LogisticRegression to classification from regression Also, add regularization parameter to SGD --- .../mllib/classification/Classification.scala | 21 +++ .../LogisticRegression.scala | 28 +-- .../LogisticRegressionGenerator.scala | 2 +- .../spark/mllib/classification/SVM.scala | 170 ++++++++++++++++++ .../mllib/classification/SVMGenerator.scala | 45 +++++ .../spark/mllib/optimization/Gradient.scala | 28 +++ .../mllib/optimization/GradientDescent.scala | 10 +- .../spark/mllib/optimization/Updater.scala | 31 +++- .../scala/spark/mllib/regression/Lasso.scala | 167 +++++++++++++++++ .../mllib/regression/LassoGenerator.scala | 44 +++++ .../LogisticRegressionSuite.scala | 2 +- .../spark/mllib/classification/SVMSuite.scala | 61 +++++++ .../spark/mllib/regression/LassoSuite.scala | 51 ++++++ 13 files changed, 642 insertions(+), 18 deletions(-) create mode 100644 mllib/src/main/scala/spark/mllib/classification/Classification.scala rename mllib/src/main/scala/spark/mllib/{regression => classification}/LogisticRegression.scala (84%) rename mllib/src/main/scala/spark/mllib/{regression => classification}/LogisticRegressionGenerator.scala (96%) create mode 100644 mllib/src/main/scala/spark/mllib/classification/SVM.scala create mode 100644 mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/Lasso.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala rename mllib/src/test/scala/spark/mllib/{regression => classification}/LogisticRegressionSuite.scala (97%) create mode 100644 mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala create mode 100644 mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/Classification.scala new file mode 100644 index 0000000000..7f1eb21079 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/classification/Classification.scala @@ -0,0 +1,21 @@ +package spark.mllib.classification + +import spark.RDD + +trait ClassificationModel { + /** + * Predict values for the given data set using the model trained. + * + * @param testData RDD representing data points to be predicted + * @return RDD[Double] where each entry contains the corresponding prediction + */ + def predict(testData: RDD[Array[Double]]): RDD[Double] + + /** + * Predict values for a single data point using the model trained. + * + * @param testData array representing a single data point + * @return Double prediction from the trained model + */ + def predict(testData: Array[Double]): Double +} diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala similarity index 84% rename from mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala rename to mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index e4db7bb9b7..f39c1ec52e 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -1,4 +1,4 @@ -package spark.mllib.regression +package spark.mllib.classification import spark.{Logging, RDD, SparkContext} import spark.mllib.optimization._ @@ -13,7 +13,7 @@ import org.jblas.DoubleMatrix class LogisticRegressionModel( val weights: DoubleMatrix, val intercept: Double, - val losses: Array[Double]) extends RegressionModel { + val losses: Array[Double]) extends ClassificationModel { override def predict(testData: spark.RDD[Array[Double]]) = { testData.map { x => @@ -29,14 +29,14 @@ class LogisticRegressionModel( } } -class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, +class LogisticRegression private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 1.0, 100) + def this() = this(1.0, 1.0, 1.0, 100) /** * Set the step size per-iteration of SGD. Default 1.0. @@ -69,7 +69,7 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, regParam, miniBatchFraction) val weightsScaled = weights.getRange(1, weights.length) val intercept = weights.get(0) @@ -96,16 +96,18 @@ object LogisticRegression { * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, stepSize: Double, + regParam: Double, miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(input) + new LogisticRegression(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -115,16 +117,18 @@ object LogisticRegression { * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. * @param numIterations Number of iterations of gradient descent to run. * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, - stepSize: Double) + stepSize: Double, + regParam: Double) : LogisticRegressionModel = { - train(input, numIterations, stepSize, 1.0) + train(input, numIterations, stepSize, regParam, 1.0) } /** @@ -141,17 +145,17 @@ object LogisticRegression { numIterations: Int) : LogisticRegressionModel = { - train(input, numIterations, 1.0, 1.0) + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { - if (args.length != 4) { - println("Usage: LogisticRegression ") + if (args.length != 5) { + println("Usage: LogisticRegression ") System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) + val model = LogisticRegression.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala index 6e7c023bac..cde1148adf 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala @@ -1,4 +1,4 @@ -package spark.mllib.regression +package spark.mllib.classification import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala new file mode 100644 index 0000000000..aceb903f1d --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -0,0 +1,170 @@ +package spark.mllib.classification + +import scala.math.signum +import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * SVM using Stochastic Gradient Descent. + */ +class SVMModel( + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends ClassificationModel { + + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => { + println("Predicting " + x) + signum(new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept) + } + } + } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + signum(dataMat.dot(this.weights) + this.intercept) + } +} + +class SVM private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, + var numIters: Int) + extends Logging { + + /** + * Construct a SVM object with default parameters + */ + def this() = this(1.0, 1.0, 1.0, 100) + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + this.stepSize = step + this + } + + /** + * Set the regularization parameter. Default 1.0. + */ + def setRegParam(param: Double) = { + this.regParam = param + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + this.numIters = iters + this + } + + def train(input: RDD[(Double, Array[Double])]): SVMModel = { + // Add a extra variable consisting of all 1.0's for the intercept. + val data = input.map { case (y, features) => + (y, Array(1.0, features:_*)) + } + + val (weights, losses) = GradientDescent.runMiniBatchSGD( + data, new HingeGradient(), new SquaredL2Updater(), stepSize, numIters, regParam, miniBatchFraction) + + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) + + val model = new SVMModel(weightsScaled, intercept, losses) + + logInfo("Final model weights " + model.weights) + logInfo("Final model intercept " + model.intercept) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + model + } +} + +/** + * Top-level methods for calling SVM. + */ +object SVM { + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : SVMModel = + { + new SVM(stepSize, regParam, miniBatchFraction, numIterations).train(input) + } + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. + * @return a SVMModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double) + : SVMModel = + { + train(input, numIterations, stepSize, regParam, 1.0) + } + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a SVMModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int) + : SVMModel = + { + train(input, numIterations, 1.0, 0.10, 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 = SVM.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala new file mode 100644 index 0000000000..a5e2837343 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala @@ -0,0 +1,45 @@ +package spark.mllib.classification + +import scala.util.Random +import scala.math.signum + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.util.MLUtils + +object LassoGenerator { + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LassoGenerator " + + " ") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 + val parts: Int = if (args.length > 4) args(4).toInt else 2 + val eps = 3 + + val sc = new SparkContext(sparkMaster, "LassoGenerator") + + val globalRnd = new Random(94720) + val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => + val rnd = new Random(42 + idx) + + val x = Array.fill[Double](nfeatures) { + rnd.nextDouble() * 2.0 - 1.0 + } + val y = signum(((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1) + (y, x) + } + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 90b0999a5e..6ffc3b128b 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -31,3 +31,31 @@ class LogisticGradient extends Gradient { (gradient, loss) } } + + +class SquaredGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + val diff: Double = data.dot(weights) - label + + val loss = 0.5 * diff * diff + val gradient = data.mul(diff) + + (gradient, loss) + } +} + + +class HingeGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + + val dotProduct = data.dot(weights) + + if (1.0 > label * dotProduct) + (data.mul(-label), 1.0 - label * dotProduct) + else + (DoubleMatrix.zeros(1,weights.length), 0.0) + } +} + diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index eff853f379..bd8489c386 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -19,6 +19,7 @@ object GradientDescent { * @param updater - Updater object that will be used to update the model. * @param stepSize - stepSize to be used during update. * @param numIters - number of iterations that SGD should be run. + * @param regParam - regularization parameter * @param miniBatchFraction - fraction of the input data set that should be used for * one iteration of SGD. Default value 1.0. * @@ -31,6 +32,7 @@ object GradientDescent { updater: Updater, stepSize: Double, numIters: Int, + regParam: Double, miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { val lossHistory = new ArrayBuffer[Double](numIters) @@ -51,10 +53,14 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - lossHistory.append(lossSum / miniBatchSize + reg_val) - val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) + val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) weights = update._1 reg_val = update._2 + lossHistory.append(lossSum / miniBatchSize + reg_val) + /*** + Xinghao: The loss here is sum of lossSum computed using the weights before applying updater, + and reg_val using weights after applying updater + ***/ } (weights, lossHistory.toArray) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index ea80bfcbfd..64c54dfb0d 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -1,5 +1,6 @@ package spark.mllib.optimization +import scala.math._ import org.jblas.DoubleMatrix abstract class Updater extends Serializable { @@ -10,18 +11,44 @@ abstract class Updater extends Serializable { * @param gradient - Column matrix of size nx1 where n is the number of features. * @param stepSize - step size across iterations * @param iter - Iteration number + * @param regParam - Regularization parameter * * @return weightsNew - Column matrix containing updated weights * @return reg_val - regularization value */ - def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) } class SimpleUpdater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int): (DoubleMatrix, Double) = { + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { val normGradient = gradient.mul(stepSize / math.sqrt(iter)) (weightsOld.sub(normGradient), 0) } } + +class L1Updater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) + val newWeights = weightsOld.sub(normGradient) + (0 until newWeights.length).foreach(i => { + val wi = newWeights.get(i) + newWeights.put(i, signum(wi) * max(0.0, abs(wi) - regParam * thisIterStepSize)) + }) + (newWeights, newWeights.norm1 * regParam) + } +} + +class SquaredL2Updater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) + val newWeights = weightsOld.sub(normGradient).div(2.0 * thisIterStepSize * regParam + 1.0) + (newWeights, pow(newWeights.norm2,2.0) * regParam) + } +} + diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala new file mode 100644 index 0000000000..de410711a2 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -0,0 +1,167 @@ +package spark.mllib.regression + +import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Lasso using Stochastic Gradient Descent. + */ +class LassoModel( + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends RegressionModel { + + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => + new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept + } + } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + dataMat.dot(this.weights) + this.intercept + } +} + +class Lasso private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, + var numIters: Int) + extends Logging { + + /** + * Construct a Lasso object with default parameters + */ + def this() = this(1.0, 1.0, 1.0, 100) + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + this.stepSize = step + this + } + + /** + * Set the regularization parameter. Default 1.0. + */ + def setRegParam(param: Double) = { + this.regParam = param + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + this.numIters = iters + this + } + + def train(input: RDD[(Double, Array[Double])]): LassoModel = { + // Add a extra variable consisting of all 1.0's for the intercept. + val data = input.map { case (y, features) => + (y, Array(1.0, features:_*)) + } + + val (weights, losses) = GradientDescent.runMiniBatchSGD( + data, new SquaredGradient(), new L1Updater(), stepSize, numIters, regParam, miniBatchFraction) + + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) + + val model = new LassoModel(weightsScaled, intercept, losses) + + logInfo("Final model weights " + model.weights) + logInfo("Final model intercept " + model.intercept) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + model + } +} + +/** + * Top-level methods for calling Lasso. + */ +object Lasso { + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : LassoModel = + { + new Lasso(stepSize, regParam, miniBatchFraction, numIterations).train(input) + } + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LassoModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double) + : LassoModel = + { + train(input, numIterations, stepSize, regParam, 1.0) + } + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LassoModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int) + : LassoModel = + { + train(input, numIterations, 1.0, 0.10, 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 = Lasso.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala new file mode 100644 index 0000000000..d2d3bb33c7 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala @@ -0,0 +1,44 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.util.MLUtils + +object LassoGenerator { + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LassoGenerator " + + " ") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 + val parts: Int = if (args.length > 4) args(4).toInt else 2 + val eps = 3 + + val sc = new SparkContext(sparkMaster, "LassoGenerator") + + val globalRnd = new Random(94720) + val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => + val rnd = new Random(42 + idx) + + val x = Array.fill[Double](nfeatures) { + rnd.nextDouble() * 2.0 - 1.0 + } + val y = ((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1 + (y, x) + } + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala similarity index 97% rename from mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala rename to mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 04d3400cb4..13612e9a4a 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.mllib.regression +package spark.mllib.classification import scala.util.Random diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala new file mode 100644 index 0000000000..e3a6681ab2 --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -0,0 +1,61 @@ +package spark.mllib.classification + +import scala.util.Random +import scala.math.signum + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + +import java.io._ + +class SVMSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + test("SVM") { + val nPoints = 10000 + val rnd = new Random(42) + + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val A = 2.0 + val B = -1.5 + val C = 1.0 + + val y = (0 until nPoints).map { i => + signum(A + B * x1(i) + C * x2(i) + 0.0*rnd.nextGaussian()) + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val writer_data = new PrintWriter(new File("svmtest.dat")) + testData.foreach(yx => { + writer_data.write(yx._1 + "") + yx._2.foreach(xi => writer_data.write("\t" + xi)) + writer_data.write("\n")}) + writer_data.close() + + val svm = new SVM().setStepSize(1.0) + .setRegParam(1.0) + .setNumIterations(100) + + val model = svm.train(testRDD) + + val yPredict = (0 until nPoints).map(i => model.predict(Array(x1(i),x2(i)))) + + val accuracy = ((y zip yPredict).map(yy => if (yy._1==yy._2) 1 else 0).reduceLeft(_+_).toDouble / nPoints.toDouble) + + assert(accuracy >= 0.90, "Accuracy (" + accuracy + ") too low") + } +} diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala new file mode 100644 index 0000000000..90fedb3e84 --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -0,0 +1,51 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + + +class LassoSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + test("Lasso") { + val nPoints = 10000 + val rnd = new Random(42) + + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val A = 2.0 + val B = -1.5 + val C = 1.0e-2 + + val y = (0 until nPoints).map { i => + A + B * x1(i) + C * x2(i) + 0.1*rnd.nextGaussian() + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val ls = new Lasso().setStepSize(1.0) + .setRegParam(0.01) + .setNumIterations(20) + + val model = ls.train(testRDD) + + val weight0 = model.weights.get(0) + val weight1 = model.weights.get(1) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + } +} From c258718606a2960649dde0a4925fcf385d617c37 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 24 Jul 2013 16:12:32 -0700 Subject: [PATCH 155/221] Fix Maven build errors after previous commits --- pom.xml | 18 ++++--- tools/pom.xml | 133 ++++++++++++++++++++++++++++++++++++++++++++------ 2 files changed, 129 insertions(+), 22 deletions(-) diff --git a/pom.xml b/pom.xml index 1fe288b92c..4cf9589b07 100644 --- a/pom.xml +++ b/pom.xml @@ -248,6 +248,16 @@ lift-json_2.9.2 2.5
        + + com.codahale.metrics + metrics-core + 3.0.0 + + + com.codahale.metrics + metrics-jvm + 3.0.0 + org.scala-lang scala-compiler @@ -267,14 +277,6 @@ org.scala-lang scalap ${scala.version} - com.codahale.metrics - metrics-core - 3.0.0 - - - com.codahale.metrics - metrics-jvm - 3.0.0 diff --git a/tools/pom.xml b/tools/pom.xml index 3dfba5808c..1125aba4f1 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,28 +20,16 @@ org.spark-project spark-parent - 0.7.4-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml org.spark-project - spark-tools_2.9.3 + spark-tools jar Spark Project Tools http://spark-project.org/ - - - org.spark-project - spark-core_2.9.3 - ${project.version} - - - org.spark-project - spark-streaming_2.9.3 - ${project.version} - - target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes @@ -60,4 +48,121 @@ + + + + hadoop1 + + + org.spark-project + spark-core + ${project.version} + hadoop1 + + + org.spark-project + spark-streaming + ${project.version} + hadoop1 + + + org.apache.hadoop + hadoop-core + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop1 + + + + + + + hadoop2 + + + org.spark-project + spark-core + ${project.version} + hadoop2 + + + org.spark-project + spark-streaming + ${project.version} + hadoop2 + + + org.apache.hadoop + hadoop-core + provided + + + org.apache.hadoop + hadoop-client + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2 + + + + + + + hadoop2-yarn + + + org.spark-project + spark-core + ${project.version} + hadoop2-yarn + + + org.spark-project + spark-streaming + ${project.version} + hadoop2-yarn + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2-yarn + + + + + + From 8e0939f5a9c0823d51d0e94185b55d4a02628a53 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:43:57 -0700 Subject: [PATCH 156/221] refactor Kryo serializer support to use chill/chill-java --- .../src/main/scala/spark/KryoSerializer.scala | 145 ++++-------------- .../scala/spark/KryoSerializerSuite.scala | 32 +--- project/SparkBuild.scala | 5 +- 3 files changed, 38 insertions(+), 144 deletions(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index ee37da7948..2ea90d41c3 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -19,24 +19,15 @@ package spark import java.io._ import java.nio.ByteBuffer -import java.nio.channels.Channels - -import scala.collection.immutable -import scala.collection.mutable - -import com.esotericsoftware.kryo._ -import com.esotericsoftware.kryo.{Serializer => KSerializer} +import com.esotericsoftware.kryo.{Kryo, KryoException} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} -import de.javakaffee.kryoserializers.KryoReflectionFactorySupport - +import com.twitter.chill.ScalaKryoInstantiator import serializer.{SerializerInstance, DeserializationStream, SerializationStream} import spark.broadcast._ import spark.storage._ -private[spark] -class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { - +private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) def writeObject[T](t: T): SerializationStream = { @@ -48,9 +39,7 @@ class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends Seria def close() { output.close() } } -private[spark] -class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { - +private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { val input = new KryoInput(inStream) def readObject[T](): T = { @@ -58,7 +47,7 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser kryo.readClassAndObject(input).asInstanceOf[T] } catch { // DeserializationStream uses the EOF exception to indicate stopping condition. - case e: com.esotericsoftware.kryo.KryoException => throw new java.io.EOFException + case _: KryoException => throw new EOFException } } @@ -69,10 +58,9 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser } private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - - val kryo = ks.kryo.get() - val output = ks.output.get() - val input = ks.input.get() + val kryo = ks.newKryo() + val output = ks.newKryoOutput() + val input = ks.newKryoInput() def serialize[T](t: T): ByteBuffer = { output.clear() @@ -108,126 +96,52 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ * serialization. */ trait KryoRegistrator { - def registerClasses(kryo: Kryo): Unit + def registerClasses(kryo: Kryo) } /** * A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]]. */ class KryoSerializer extends spark.serializer.Serializer with Logging { + private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 - val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + def newKryoOutput() = new KryoOutput(bufferSize) - val kryo = new ThreadLocal[Kryo] { - override def initialValue = createKryo() - } + def newKryoInput() = new KryoInput(bufferSize) - val output = new ThreadLocal[KryoOutput] { - override def initialValue = new KryoOutput(bufferSize) - } - - val input = new ThreadLocal[KryoInput] { - override def initialValue = new KryoInput(bufferSize) - } - - def createKryo(): Kryo = { - val kryo = new KryoReflectionFactorySupport() + def newKryo(): Kryo = { + val instantiator = new ScalaKryoInstantiator + val kryo = instantiator.newKryo() + val classLoader = Thread.currentThread.getContextClassLoader // Register some commonly used classes val toRegister: Seq[AnyRef] = Seq( - // Arrays - Array(1), Array(1.0), Array(1.0f), Array(1L), Array(""), Array(("", "")), - Array(new java.lang.Object), Array(1.toByte), Array(true), Array('c'), - // Specialized Tuple2s - ("", ""), ("", 1), (1, 1), (1.0, 1.0), (1L, 1L), - (1, 1.0), (1.0, 1), (1L, 1.0), (1.0, 1L), (1, 1L), (1L, 1), - // Scala collections - List(1), mutable.ArrayBuffer(1), - // Options and Either - Some(1), Left(1), Right(1), - // Higher-dimensional tuples - (1, 1, 1), (1, 1, 1, 1), (1, 1, 1, 1, 1), - None, ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY, PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY), GotBlock("1", ByteBuffer.allocate(1)), GetBlock("1") ) - for (obj <- toRegister) { - kryo.register(obj.getClass) - } + + for (obj <- toRegister) kryo.register(obj.getClass) // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) - // Register some commonly used Scala singleton objects. Because these - // are singletons, we must return the exact same local object when we - // deserialize rather than returning a clone as FieldSerializer would. - class SingletonSerializer[T](obj: T) extends KSerializer[T] { - override def write(kryo: Kryo, output: KryoOutput, obj: T) {} - override def read(kryo: Kryo, input: KryoInput, cls: java.lang.Class[T]): T = obj - } - kryo.register(None.getClass, new SingletonSerializer[AnyRef](None)) - kryo.register(Nil.getClass, new SingletonSerializer[AnyRef](Nil)) - - // Register maps with a special serializer since they have complex internal structure - class ScalaMapSerializer(buildMap: Array[(Any, Any)] => scala.collection.Map[Any, Any]) - extends KSerializer[Array[(Any, Any)] => scala.collection.Map[Any, Any]] { - - //hack, look at https://groups.google.com/forum/#!msg/kryo-users/Eu5V4bxCfws/k-8UQ22y59AJ - private final val FAKE_REFERENCE = new Object() - override def write( - kryo: Kryo, - output: KryoOutput, - obj: Array[(Any, Any)] => scala.collection.Map[Any, Any]) { - val map = obj.asInstanceOf[scala.collection.Map[Any, Any]] - output.writeInt(map.size) - for ((k, v) <- map) { - kryo.writeClassAndObject(output, k) - kryo.writeClassAndObject(output, v) - } - } - override def read ( - kryo: Kryo, - input: KryoInput, - cls: Class[Array[(Any, Any)] => scala.collection.Map[Any, Any]]) - : Array[(Any, Any)] => scala.collection.Map[Any, Any] = { - kryo.reference(FAKE_REFERENCE) - val size = input.readInt() - val elems = new Array[(Any, Any)](size) - for (i <- 0 until size) { - val k = kryo.readClassAndObject(input) - val v = kryo.readClassAndObject(input) - elems(i)=(k,v) - } - buildMap(elems).asInstanceOf[Array[(Any, Any)] => scala.collection.Map[Any, Any]] - } - } - kryo.register(mutable.HashMap().getClass, new ScalaMapSerializer(mutable.HashMap() ++ _)) - // TODO: add support for immutable maps too; this is more annoying because there are many - // subclasses of immutable.Map for small maps (with <= 4 entries) - val map1 = Map[Any, Any](1 -> 1) - val map2 = Map[Any, Any](1 -> 1, 2 -> 2) - val map3 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3) - val map4 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4) - val map5 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4, 5 -> 5) - kryo.register(map1.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map2.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map3.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map4.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map5.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - // Allow the user to register their own classes by setting spark.kryo.registrator - val regCls = System.getProperty("spark.kryo.registrator") - if (regCls != null) { - logInfo("Running user registrator: " + regCls) - val classLoader = Thread.currentThread.getContextClassLoader - val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] - reg.registerClasses(kryo) + try { + Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => + logDebug("Running user registrator: " + regCls) + val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] + reg.registerClasses(kryo) + } + } catch { + case _: Exception => println("Failed to register spark.kryo.registrator") } + kryo.setClassLoader(classLoader) + // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) @@ -235,7 +149,6 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { } def newInstance(): SerializerInstance = { - this.kryo.get().setClassLoader(Thread.currentThread().getContextClassLoader) new KryoSerializerInstance(this) } -} +} \ No newline at end of file diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index c3323dcbb3..cb59bb9b65 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -1,30 +1,10 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - package spark import scala.collection.mutable -import scala.collection.immutable import org.scalatest.FunSuite import com.esotericsoftware.kryo._ -import SparkContext._ - class KryoSerializerSuite extends FunSuite { test("basic types") { val ser = (new KryoSerializer).newInstance() @@ -53,6 +33,7 @@ class KryoSerializerSuite extends FunSuite { check(Array(true, false, true)) check(Array('a', 'b', 'c')) check(Array[Int]()) + check(Array(Array("1", "2"), Array("1", "2", "3", "4"))) } test("pairs") { @@ -99,11 +80,10 @@ class KryoSerializerSuite extends FunSuite { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) - check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) } test("custom registrator") { - import spark.test._ + import KryoTest._ System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) val ser = (new KryoSerializer).newInstance() @@ -123,14 +103,14 @@ class KryoSerializerSuite extends FunSuite { val hashMap = new java.util.HashMap[String, String] hashMap.put("foo", "bar") check(hashMap) - + System.clearProperty("spark.kryo.registrator") } } -package test { +object KryoTest { case class CaseClass(i: Int, s: String) {} - + class ClassWithNoArgConstructor { var x: Int = 0 override def equals(other: Any) = other match { @@ -154,4 +134,4 @@ package test { k.register(classOf[java.util.HashMap[_, _]]) } } -} +} \ No newline at end of file diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d4d70afdd5..9920e00a67 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -170,7 +170,6 @@ object SparkBuild extends Build { "com.ning" % "compress-lzf" % "0.8.4", "org.ow2.asm" % "asm" % "4.0", "com.google.protobuf" % "protobuf-java" % "2.4.1", - "de.javakaffee" % "kryo-serializers" % "0.22", "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty), @@ -181,7 +180,9 @@ object SparkBuild extends Build { "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", - "com.codahale.metrics" % "metrics-jvm" % "3.0.0" + "com.codahale.metrics" % "metrics-jvm" % "3.0.0", + "com.twitter" % "chill_2.9.3" % "0.3.0", + "com.twitter" % "chill-java" % "0.3.0" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { From a1c515fb02646c857c607949120c7d3cb29f4dce Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:50:32 -0700 Subject: [PATCH 157/221] add copyright back in --- .../test/scala/spark/KryoSerializerSuite.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index cb59bb9b65..f6e3eec546 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable From fc4b025314232fbeeaacb84ec3e2491e9c4ddd1b Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:53:15 -0700 Subject: [PATCH 158/221] add test --- core/src/test/scala/spark/KryoSerializerSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index f6e3eec546..30d2d5282b 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -97,6 +97,7 @@ class KryoSerializerSuite extends FunSuite { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) + check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) } test("custom registrator") { From 30a369a808cdc16effe9c5755c74fc5a3f3d7cac Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:55:48 -0700 Subject: [PATCH 159/221] update pom.xml --- pom.xml | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 4cf9589b07..469dc64329 100644 --- a/pom.xml +++ b/pom.xml @@ -193,9 +193,14 @@ 2.4.1 - de.javakaffee - kryo-serializers - 0.22 + com.twitter + chill + 0.3.0 + + + com.twitter + chill-java + 0.3.0 com.typesafe.akka From e56aa75de0f3c00e9942f0863c0fb8c57aab5321 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 22:08:09 -0700 Subject: [PATCH 160/221] fix wrapping --- core/src/main/scala/spark/KryoSerializer.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index 2ea90d41c3..eeb2993d8a 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -27,7 +27,8 @@ import serializer.{SerializerInstance, DeserializationStream, SerializationStrea import spark.broadcast._ import spark.storage._ -private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { +private[spark] +class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) def writeObject[T](t: T): SerializationStream = { @@ -39,7 +40,8 @@ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream def close() { output.close() } } -private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { +private[spark] +class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { val input = new KryoInput(inStream) def readObject[T](): T = { From e2421c13117c7d9cc950da8c9de4903d0ea49c43 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Jul 2013 00:05:43 -0700 Subject: [PATCH 161/221] Update Chill reference in pom.xml too --- core/pom.xml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 237d988712..8c740e3887 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -57,8 +57,14 @@ protobuf-java - de.javakaffee - kryo-serializers + com.twitter + chill + 0.3.0 + + + com.twitter + chill-java + 0.3.0 com.typesafe.akka From 8eb8b52997da56acb3500fa48aac1ab9a3dfdddf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Jul 2013 08:58:02 -0700 Subject: [PATCH 162/221] Fix Chill version in Maven --- core/pom.xml | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 8c740e3887..f0c936c86a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -58,7 +58,7 @@ com.twitter - chill + chill_2.9.3 0.3.0 diff --git a/pom.xml b/pom.xml index 469dc64329..44729bd422 100644 --- a/pom.xml +++ b/pom.xml @@ -194,7 +194,7 @@ com.twitter - chill + chill_2.9.3 0.3.0 From a6de90c927e5569e30a7bc6188253bd14c32c3a9 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Thu, 25 Jul 2013 01:01:50 -0700 Subject: [PATCH 163/221] For standalone mode, get JAVA_HOME, SPARK_JAVA_OPTS, SPARK_LIBRARY_PATH from application env, not worker env --- .../main/scala/spark/deploy/worker/ExecutorRunner.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 8b51ff1c3a..47d3390928 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -51,6 +51,9 @@ private[spark] class ExecutorRunner( var process: Process = null var shutdownHook: Thread = null + private def getAppEnv(key: String): Option[String] = + appDesc.command.environment.get(key).orElse(Option(getenv(key))) + def start() { workerThread = new Thread("ExecutorRunner for " + fullId) { override def run() { fetchAndRunExecutor() } @@ -95,7 +98,7 @@ private[spark] class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = appDesc.command - val runner = Option(getenv("JAVA_HOME")).map(_ + "/bin/java").getOrElse("java") + val runner = getAppEnv("JAVA_HOME").map(_ + "/bin/java").getOrElse("java") // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++ @@ -107,10 +110,10 @@ private[spark] class ExecutorRunner( * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val libraryOpts = Option(getenv("SPARK_LIBRARY_PATH")) + val libraryOpts = getAppEnv("SPARK_LIBRARY_PATH") .map(p => List("-Djava.library.path=" + p)) .getOrElse(Nil) - val userOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil) + val userOpts = getAppEnv("SPARK_JAVA_OPTS").map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") // Figure out our classpath with the external compute-classpath script From d4bbc8bd251937a778658e09791b0416fd54336e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 25 Jul 2013 15:59:52 -0700 Subject: [PATCH 164/221] Shows totals for shuffle data and CPU time in Stage, homepage overviews including active time --- .../scala/spark/ui/exec/ExecutorsUI.scala | 10 ++--- .../main/scala/spark/ui/jobs/IndexPage.scala | 28 +++++++++++++- .../scala/spark/ui/jobs/JobProgressUI.scala | 38 ++++++++++++++++--- .../main/scala/spark/ui/jobs/StagePage.scala | 26 ++++++++++++- 4 files changed, 90 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index db1c902955..e569c04195 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -113,7 +113,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, HashSet[Long]]() + val executorToTasksActive = HashMap[String, HashSet[TaskInfo]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = @@ -122,8 +122,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId if (!executorToTasksActive.contains(eid)) - executorToTasksActive(eid) = HashSet[Long]() - executorToTasksActive(eid) += taskStart.taskInfo.taskId + executorToTasksActive(eid) = HashSet[TaskInfo]() + executorToTasksActive(eid) += taskStart.taskInfo val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -133,8 +133,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId if (!executorToTasksActive.contains(eid)) - executorToTasksActive(eid) = HashSet[Long]() - executorToTasksActive(eid) -= taskEnd.taskInfo.taskId + executorToTasksActive(eid) = HashSet[TaskInfo]() + executorToTasksActive(eid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index f31af3cda6..41186fd9bc 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -28,6 +28,7 @@ import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ import spark.storage.StorageLevel +import spark.Utils /** Page showing list of all ongoing and recently finished stages */ private[spark] class IndexPage(parent: JobProgressUI) { @@ -39,6 +40,13 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq + var activeTime = 0L + listener.stageToTasksActive.foreach {s => + s._2.foreach { t => + activeTime += t.timeRunning(System.currentTimeMillis()) + } + } + /** Special table which merges two header cells. */ def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { @@ -57,11 +65,29 @@ private[spark] class IndexPage(parent: JobProgressUI) {
        } + val summary: NodeSeq = +
        +
          +
        • + CPU time: + {parent.formatDuration(listener.totalTime + activeTime)} +
        • +
        • + Shuffle read: + {Utils.memoryBytesToString(listener.totalShuffleRead)} +
        • +
        • + Shuffle write: + {Utils.memoryBytesToString(listener.totalShuffleWrite)} +
        • +
        +
        val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) val completedStageTable = stageTable(stageRow, completedStages) val failedStageTable: NodeSeq = stageTable(stageRow, failedStages) - val content =

        Active Stages

        ++ activeStageTable ++ + val content = summary ++ +

        Active Stages

        ++ activeStageTable ++

        Completed Stages

        ++ completedStageTable ++

        Failed Stages

        ++ failedStageTable diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 6e332415db..3cc38ea403 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -65,7 +65,14 @@ private[spark] class JobProgressListener extends SparkListener { val completedStages = ListBuffer[Stage]() val failedStages = ListBuffer[Stage]() - val stageToTasksActive = HashMap[Int, HashSet[Long]]() + var totalTime = 0L + var totalShuffleRead = 0L + var totalShuffleWrite = 0L + + val stageToTime = HashMap[Int, Long]() + val stageToShuffleRead = HashMap[Int, Long]() + val stageToShuffleWrite = HashMap[Int, Long]() + val stageToTasksActive = HashMap[Int, HashSet[TaskInfo]]() val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = @@ -97,8 +104,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { val sid = taskStart.task.stageId if (!stageToTasksActive.contains(sid)) - stageToTasksActive(sid) = HashSet[Long]() - stageToTasksActive(sid) += taskStart.taskInfo.taskId + stageToTasksActive(sid) = HashSet[TaskInfo]() + stageToTasksActive(sid) += taskStart.taskInfo val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -108,8 +115,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId if (!stageToTasksActive.contains(sid)) - stageToTasksActive(sid) = HashSet[Long]() - stageToTasksActive(sid) -= taskEnd.taskInfo.taskId + stageToTasksActive(sid) = HashSet[TaskInfo]() + stageToTasksActive(sid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -119,6 +126,27 @@ private[spark] class JobProgressListener extends SparkListener { stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 (None, Some(taskEnd.taskMetrics)) } + + if (!stageToTime.contains(sid)) + stageToTime(sid) = 0L + val time = metrics.map(m => m.executorRunTime).getOrElse(1) + stageToTime(sid) += time + totalTime += time + + if (!stageToShuffleRead.contains(sid)) + stageToShuffleRead(sid) = 0L + val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => + s.remoteBytesRead).getOrElse(0L) + stageToShuffleRead(sid) += shuffleRead + totalShuffleRead += shuffleRead + + if (!stageToShuffleWrite.contains(sid)) + stageToShuffleWrite(sid) = 0L + val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => + s.shuffleBytesWritten).getOrElse(0L) + stageToShuffleWrite(sid) += shuffleWrite + totalShuffleWrite += shuffleWrite + val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList -= ((taskEnd.taskInfo, None, None)) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 654f347723..41d12b1ef4 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -52,6 +52,29 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleRead = listener.hasShuffleRead(stageId) val shuffleWrite = listener.hasShuffleWrite(stageId) + var activeTime = 0L + listener.stageToTasksActive(stageId).foreach { t => + activeTime += t.timeRunning(System.currentTimeMillis()) + } + + val summary = +
        +
          +
        • + CPU time: + {parent.formatDuration(listener.stageToTime(stageId) + activeTime)} +
        • +
        • + Shuffle read: + {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))} +
        • +
        • + Shuffle write: + {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} +
        • +
        +
        + val taskHeaders: Seq[String] = Seq("Task ID", "Status", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ @@ -98,7 +121,8 @@ private[spark] class StagePage(parent: JobProgressUI) { } val content = -

        Summary Metrics

        ++ summaryTable.getOrElse(Nil) ++

        Tasks

        ++ taskTable; + summary ++

        Summary Metrics

        ++ summaryTable.getOrElse(Nil) ++ +

        Tasks

        ++ taskTable; headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } From 22faeab261302d1506321993ecacf9f9ab9aa5f6 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 25 Jul 2013 17:14:18 -0700 Subject: [PATCH 165/221] Split Shuffle Activity overview column for read/write --- .../main/scala/spark/ui/jobs/IndexPage.scala | 25 ++++++++++++------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 41186fd9bc..9bfb9b9041 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -56,7 +56,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { Submitted Duration Tasks: Complete/Total - Shuffle Activity + Shuffle Read + Shuffle Write Stored RDD @@ -120,13 +121,18 @@ private[spark] class IndexPage(parent: JobProgressUI) { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" } - val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) - val shuffleInfo = (read, write) match { - case (true, true) => "Read/Write" - case (true, false) => "Read" - case (false, true) => "Write" - case _ => "" - } + + val shuffleRead = + if (!listener.hasShuffleRead(s.id)) + "" + else + Utils.memoryBytesToString(listener.stageToShuffleRead(s.id)) + val shuffleWrite = + if (!listener.hasShuffleWrite(s.id)) + "" + else + Utils.memoryBytesToString(listener.stageToShuffleWrite(s.id)) + val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions @@ -143,7 +149,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { case _ => }} - {shuffleInfo} + {shuffleRead} + {shuffleWrite} {if (s.rdd.getStorageLevel != StorageLevel.NONE) { {Option(s.rdd.name).getOrElse(s.rdd.id)} From 3fbe9eaac08c13527e64e4fe1ce2748838707562 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 26 Jul 2013 11:51:38 -0700 Subject: [PATCH 166/221] Displys shuffle read/write only if exists, wraps if statements, trims old vals, grabs current time once --- .../scala/spark/ui/exec/ExecutorsUI.scala | 6 ++- .../main/scala/spark/ui/jobs/IndexPage.scala | 27 ++++++------ .../scala/spark/ui/jobs/JobProgressUI.scala | 41 ++++++++++++++----- .../main/scala/spark/ui/jobs/StagePage.scala | 20 +++++---- 4 files changed, 62 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index e569c04195..b15da81f7c 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -121,8 +121,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) + if (!executorToTasksActive.contains(eid)) { executorToTasksActive(eid) = HashSet[TaskInfo]() + } executorToTasksActive(eid) += taskStart.taskInfo val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) @@ -132,8 +133,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) + if (!executorToTasksActive.contains(eid)) { executorToTasksActive(eid) = HashSet[TaskInfo]() + } executorToTasksActive(eid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 9bfb9b9041..b862c3539c 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -39,12 +39,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq + val now = System.currentTimeMillis() var activeTime = 0L - listener.stageToTasksActive.foreach {s => - s._2.foreach { t => - activeTime += t.timeRunning(System.currentTimeMillis()) - } + for (tasks <- listener.stageToTasksActive.values; t <- tasks) { + activeTime += t.timeRunning(now) } /** Special table which merges two header cells. */ @@ -73,14 +72,18 @@ private[spark] class IndexPage(parent: JobProgressUI) { CPU time: {parent.formatDuration(listener.totalTime + activeTime)} -
      • - Shuffle read: - {Utils.memoryBytesToString(listener.totalShuffleRead)} -
      • -
      • - Shuffle write: - {Utils.memoryBytesToString(listener.totalShuffleWrite)} -
      • + {if (listener.totalShuffleRead > 0) +
      • + Shuffle read: + {Utils.memoryBytesToString(listener.totalShuffleRead)} +
      • + } + {if (listener.totalShuffleWrite > 0) +
      • + Shuffle write: + {Utils.memoryBytesToString(listener.totalShuffleWrite)} +
      • + }
      val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 3cc38ea403..2fb4575551 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -93,6 +93,12 @@ private[spark] class JobProgressListener extends SparkListener { val toRemove = RETAINED_STAGES / 10 stages.takeRight(toRemove).foreach( s => { stageToTaskInfos.remove(s.id) + stageToTime.remove(s.id) + stageToShuffleRead.remove(s.id) + stageToShuffleWrite.remove(s.id) + stageToTasksActive.remove(s.id) + stageToTasksComplete.remove(s.id) + stageToTasksFailed.remove(s.id) }) stages.trimEnd(toRemove) } @@ -103,8 +109,9 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { val sid = taskStart.task.stageId - if (!stageToTasksActive.contains(sid)) + if (!stageToTasksActive.contains(sid)) { stageToTasksActive(sid) = HashSet[TaskInfo]() + } stageToTasksActive(sid) += taskStart.taskInfo val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) @@ -114,8 +121,9 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - if (!stageToTasksActive.contains(sid)) + if (!stageToTasksActive.contains(sid)) { stageToTasksActive(sid) = HashSet[TaskInfo]() + } stageToTasksActive(sid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { @@ -127,23 +135,36 @@ private[spark] class JobProgressListener extends SparkListener { (None, Some(taskEnd.taskMetrics)) } - if (!stageToTime.contains(sid)) + if (!stageToTime.contains(sid)) { stageToTime(sid) = 0L - val time = metrics.map(m => m.executorRunTime).getOrElse(1) + } + val time = if (metrics.isDefined) metrics.map(m => m.executorRunTime).getOrElse(0) else 0 stageToTime(sid) += time totalTime += time - if (!stageToShuffleRead.contains(sid)) + if (!stageToShuffleRead.contains(sid)) { stageToShuffleRead(sid) = 0L - val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => - s.remoteBytesRead).getOrElse(0L) + } + val shuffleRead = + if (metrics.isDefined) { + metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead).getOrElse(0L) + } + else { + 0L + } stageToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead - if (!stageToShuffleWrite.contains(sid)) + if (!stageToShuffleWrite.contains(sid)) { stageToShuffleWrite(sid) = 0L - val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => - s.shuffleBytesWritten).getOrElse(0L) + } + val shuffleWrite = + if (metrics.isDefined) { + metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleBytesWritten).getOrElse(0L) + } + else { + 0L + } stageToShuffleWrite(sid) += shuffleWrite totalShuffleWrite += shuffleWrite diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 41d12b1ef4..266f180a50 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -64,14 +64,18 @@ private[spark] class StagePage(parent: JobProgressUI) { CPU time: {parent.formatDuration(listener.stageToTime(stageId) + activeTime)} -
    • - Shuffle read: - {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))} -
    • -
    • - Shuffle write: - {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} -
    • + {if (listener.hasShuffleRead(stageId)) +
    • + Shuffle read: + {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))} +
    • + } + {if (listener.hasShuffleWrite(stageId)) +
    • + Shuffle write: + {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} +
    • + }
    From 743fc4e7aa8a2ca4edbe731bbefb2127d5d1a7d4 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 14:35:17 -0700 Subject: [PATCH 167/221] Fix Bug in Partition Pruning, index of Pruned Partitions should inherit from parent --- .../scala/spark/rdd/PartitionPruningRDD.scala | 6 ++-- .../spark/PartitionPruningRDDSuite.scala | 28 +++++++++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 core/src/test/scala/spark/PartitionPruningRDDSuite.scala diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 41ff62dd22..6fe004a009 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -16,8 +16,9 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo extends NarrowDependency[T](rdd) { @transient - val partitions: Array[Partition] = rdd.partitions.filter(s => partitionFilterFunc(s.index)) - .zipWithIndex.map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } + val partitions: Array[Partition] = rdd.partitions. + zipWithIndex.filter(s => partitionFilterFunc(s._2)). + map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } @@ -39,6 +40,7 @@ class PartitionPruningRDD[T: ClassManifest]( override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions + } diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala new file mode 100644 index 0000000000..a0e6413160 --- /dev/null +++ b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala @@ -0,0 +1,28 @@ +package spark + +import org.scalatest.FunSuite +import spark.SparkContext._ +import spark.rdd.PartitionPruningRDD + + +class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { + + test("Pruned Partitions inherit locality prefs correctly") { + class TestPartition(i: Int) extends Partition { + def index = i + } + val rdd = new RDD[Int](sc, Nil) { + override protected def getPartitions = { + Array[Partition]( + new TestPartition(1), + new TestPartition(2), + new TestPartition(3)) + } + def compute(split: Partition, context: TaskContext) = {Iterator()} + } + val prunedRDD = PartitionPruningRDD.create(rdd, {x => if (x==2) true else false}) + println(prunedRDD.partitions.length) + val p = prunedRDD.partitions(0) + assert(p.index == 2) + } +} \ No newline at end of file From 822aac8f5afd1342bb1cbadd7a2c2821bda81988 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 15:10:32 -0700 Subject: [PATCH 168/221] Indentation --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 1 - core/src/test/scala/spark/PartitionPruningRDDSuite.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 6fe004a009..04e22574af 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -40,7 +40,6 @@ class PartitionPruningRDD[T: ClassManifest]( override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions - } diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala index a0e6413160..4fd1f55a45 100644 --- a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala @@ -25,4 +25,4 @@ class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { val p = prunedRDD.partitions(0) assert(p.index == 2) } -} \ No newline at end of file +} From 72cf7ec0e5736d8cc5a691b36c78a79c9a4bf118 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 15:16:41 -0700 Subject: [PATCH 169/221] Indentation --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 04e22574af..eaac134ef4 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -16,9 +16,9 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo extends NarrowDependency[T](rdd) { @transient - val partitions: Array[Partition] = rdd.partitions. - zipWithIndex.filter(s => partitionFilterFunc(s._2)). - map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } + val partitions: Array[Partition] = rdd.partitions.zipWithIndex + .filter(s => partitionFilterFunc(s._2)) + .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } From 392d7474fde6f18bcd9cece5c4ed057ac1d24485 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 15:23:15 -0700 Subject: [PATCH 170/221] Code review --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 4 ++-- core/src/test/scala/spark/PartitionPruningRDDSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index eaac134ef4..fa2f5c2611 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -17,8 +17,8 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo @transient val partitions: Array[Partition] = rdd.partitions.zipWithIndex - .filter(s => partitionFilterFunc(s._2)) - .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } + .filter(s => partitionFilterFunc(s._2)) + .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala index 4fd1f55a45..88352b639f 100644 --- a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala @@ -21,8 +21,8 @@ class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { def compute(split: Partition, context: TaskContext) = {Iterator()} } val prunedRDD = PartitionPruningRDD.create(rdd, {x => if (x==2) true else false}) - println(prunedRDD.partitions.length) val p = prunedRDD.partitions(0) assert(p.index == 2) + assert(prunedRDD.partitions.length == 1) } } From 3fc64089032707393cc016b17ccd3dade20f79fe Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 26 Jul 2013 16:10:20 -0700 Subject: [PATCH 171/221] Added missing scalatest dependency --- tools/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tools/pom.xml b/tools/pom.xml index 1125aba4f1..878eb82f18 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -30,6 +30,14 @@ Spark Project Tools http://spark-project.org/ + + + org.scalatest + scalatest_${scala.version} + test + + + target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes From bd4cc52e309667296ef60557e17e8a7263c51fd8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 26 Jul 2013 17:23:18 -0700 Subject: [PATCH 172/221] Made metrics Option instead of Some, fixed NullPointerException --- .../scala/spark/ui/exec/ExecutorsUI.scala | 2 +- .../scala/spark/ui/jobs/JobProgressUI.scala | 22 +++++-------------- 2 files changed, 7 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index b15da81f7c..80d00c6873 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -144,7 +144,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { (Some(e), e.metrics) case _ => executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - (None, Some(taskEnd.taskMetrics)) + (None, Option(taskEnd.taskMetrics)) } val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 2fb4575551..e7fbff7f73 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -132,39 +132,29 @@ private[spark] class JobProgressListener extends SparkListener { (Some(e), e.metrics) case _ => stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - (None, Some(taskEnd.taskMetrics)) + (None, Option(taskEnd.taskMetrics)) } if (!stageToTime.contains(sid)) { stageToTime(sid) = 0L } - val time = if (metrics.isDefined) metrics.map(m => m.executorRunTime).getOrElse(0) else 0 + val time = metrics.map(m => m.executorRunTime).getOrElse(0) stageToTime(sid) += time totalTime += time if (!stageToShuffleRead.contains(sid)) { stageToShuffleRead(sid) = 0L } - val shuffleRead = - if (metrics.isDefined) { - metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead).getOrElse(0L) - } - else { - 0L - } + val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => + s.remoteBytesRead).getOrElse(0L) stageToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead if (!stageToShuffleWrite.contains(sid)) { stageToShuffleWrite(sid) = 0L } - val shuffleWrite = - if (metrics.isDefined) { - metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleBytesWritten).getOrElse(0L) - } - else { - 0L - } + val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => + s.shuffleBytesWritten).getOrElse(0L) stageToShuffleWrite(sid) += shuffleWrite totalShuffleWrite += shuffleWrite From f74a03c6d82f47e004d32de5bd4e17be91d35888 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 17:29:44 -0700 Subject: [PATCH 173/221] Multiple changes - Changed LogisticRegression regularization parameter to 0 - Removed println from SVM predict function - Fixed "Lasso" -> "SVM" in SVMGenerator - Added comment in Updater.scala to indicate L1 regularization leads to soft thresholding proximal function --- .../spark/mllib/classification/LogisticRegression.scala | 4 ++-- mllib/src/main/scala/spark/mllib/classification/SVM.scala | 4 +--- .../scala/spark/mllib/classification/SVMGenerator.scala | 6 +++--- mllib/src/main/scala/spark/mllib/optimization/Updater.scala | 3 +++ 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index f39c1ec52e..876197a14f 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -36,7 +36,7 @@ class LogisticRegression private (var stepSize: Double, var regParam: Double, va /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 1.0, 1.0, 100) + def this() = this(1.0, 0.0, 1.0, 100) /** * Set the step size per-iteration of SGD. Default 1.0. @@ -145,7 +145,7 @@ object LogisticRegression { numIterations: Int) : LogisticRegressionModel = { - train(input, numIterations, 1.0, 1.0, 1.0) + train(input, numIterations, 1.0, 0.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index aceb903f1d..c8da8b7c29 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -16,11 +16,9 @@ class SVMModel( val losses: Array[Double]) extends ClassificationModel { override def predict(testData: spark.RDD[Array[Double]]) = { - testData.map { x => { - println("Predicting " + x) + testData.map { x => signum(new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept) } - } } override def predict(testData: Array[Double]): Double = { diff --git a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala index a5e2837343..029f262660 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala @@ -8,11 +8,11 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.util.MLUtils -object LassoGenerator { +object SVMGenerator { def main(args: Array[String]) { if (args.length != 5) { - println("Usage: LassoGenerator " + + println("Usage: SVMGenerator " + " ") System.exit(1) } @@ -24,7 +24,7 @@ object LassoGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 3 - val sc = new SparkContext(sparkMaster, "LassoGenerator") + val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index 64c54dfb0d..26f06e86dc 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -28,6 +28,9 @@ class SimpleUpdater extends Updater { } } +/** +L1 regularization -- corresponding proximal operator is the soft-thresholding function +**/ class L1Updater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { From f0a1f95228bdb3f2f4c8a869a96c8df1a066edf3 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 17:36:14 -0700 Subject: [PATCH 174/221] Rename LogisticRegression, SVM and Lasso to *_LocalRandomSGD --- .../spark/mllib/classification/LogisticRegression.scala | 8 ++++---- mllib/src/main/scala/spark/mllib/classification/SVM.scala | 8 ++++---- mllib/src/main/scala/spark/mllib/regression/Lasso.scala | 8 ++++---- .../mllib/classification/LogisticRegressionSuite.scala | 4 ++-- .../test/scala/spark/mllib/classification/SVMSuite.scala | 4 ++-- .../test/scala/spark/mllib/regression/LassoSuite.scala | 4 ++-- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 876197a14f..243a346364 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -29,7 +29,7 @@ class LogisticRegressionModel( } } -class LogisticRegression private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -86,7 +86,7 @@ class LogisticRegression private (var stepSize: Double, var regParam: Double, va /** * Top-level methods for calling Logistic Regression. */ -object LogisticRegression { +object LogisticRegression_LocalRandomSGD { /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number @@ -107,7 +107,7 @@ object LogisticRegression { miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new LogisticRegression_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -155,7 +155,7 @@ object LogisticRegression { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegression.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LogisticRegression_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index c8da8b7c29..bf10493bf5 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -27,7 +27,7 @@ class SVMModel( } } -class SVM private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -92,7 +92,7 @@ class SVM private (var stepSize: Double, var regParam: Double, var miniBatchFrac /** * Top-level methods for calling SVM. */ -object SVM { +object SVM_LocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number @@ -113,7 +113,7 @@ object SVM { miniBatchFraction: Double) : SVMModel = { - new SVM(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -161,7 +161,7 @@ object SVM { } val sc = new SparkContext(args(0), "SVM") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = SVM.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = SVM_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index de410711a2..bb2305c811 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -26,7 +26,7 @@ class LassoModel( } } -class Lasso private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -91,7 +91,7 @@ class Lasso private (var stepSize: Double, var regParam: Double, var miniBatchFr /** * Top-level methods for calling Lasso. */ -object Lasso { +object Lasso_LocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number @@ -112,7 +112,7 @@ object Lasso { miniBatchFraction: Double) : LassoModel = { - new Lasso(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -160,7 +160,7 @@ object Lasso { } val sc = new SparkContext(args(0), "Lasso") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = Lasso.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = Lasso_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 13612e9a4a..5741906a14 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -18,7 +18,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { } // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { + test("LogisticRegression_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -45,7 +45,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(10.0) + val lr = new LogisticRegression_LocalRandomSGD().setStepSize(10.0) .setNumIterations(20) val model = lr.train(testRDD) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index e3a6681ab2..50cf260f49 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -19,7 +19,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("SVM") { + test("SVM_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -46,7 +46,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { writer_data.write("\n")}) writer_data.close() - val svm = new SVM().setStepSize(1.0) + val svm = new SVM_LocalRandomSGD().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100) diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 90fedb3e84..9836ac54c1 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -17,7 +17,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("Lasso") { + test("Lasso_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -36,7 +36,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val ls = new Lasso().setStepSize(1.0) + val ls = new Lasso_LocalRandomSGD().setStepSize(1.0) .setRegParam(0.01) .setNumIterations(20) From 10fd3949e6ccfca896ccfefe5895fda7f40944cf Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 17:49:11 -0700 Subject: [PATCH 175/221] Making ClassificationModel serializable --- .../main/scala/spark/mllib/classification/Classification.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/Classification.scala index 7f1eb21079..96d7a54f18 100644 --- a/mllib/src/main/scala/spark/mllib/classification/Classification.scala +++ b/mllib/src/main/scala/spark/mllib/classification/Classification.scala @@ -2,7 +2,7 @@ package spark.mllib.classification import spark.RDD -trait ClassificationModel { +trait ClassificationModel extends Serializable { /** * Predict values for the given data set using the model trained. * From b0bbc7f6a8da8e4c8e4e5bb656d7c7eed9b24511 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 18:57:39 -0700 Subject: [PATCH 176/221] Resolve conflicts with master, removed regParam for LogisticRegression --- .../classification/LogisticRegression.scala | 120 +++++++++++++++--- .../spark/mllib/classification/SVM.scala | 96 ++++++++++++-- .../spark/mllib/optimization/Gradient.scala | 18 ++- .../mllib/optimization/GradientDescent.scala | 37 ++++-- .../scala/spark/mllib/regression/Lasso.scala | 97 ++++++++++++-- .../LogisticRegressionSuite.scala | 108 ++++++++++++++-- 6 files changed, 412 insertions(+), 64 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 243a346364..40b96fbe3a 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.classification import spark.{Logging, RDD, SparkContext} @@ -11,32 +28,39 @@ import org.jblas.DoubleMatrix * Based on Matlab code written by John Duchi. */ class LogisticRegressionModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, - val losses: Array[Double]) extends ClassificationModel { + val stochasticLosses: Array[Double]) extends ClassificationModel { + + // Create a column vector that can be used for predictions + private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) override def predict(testData: spark.RDD[Array[Double]]) = { + // A small optimization to avoid serializing the entire model. Only the weightsMatrix + // and intercept is needed. + val localWeights = weightsMatrix + val localIntercept = intercept testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept 1.0/ (1.0 + math.exp(margin * -1)) } } override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - val margin = dataMat.mmul(this.weights).get(0) + this.intercept + val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } -class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 0.0, 1.0, 100) + def this() = this(1.0, 1.0, 100) /** * Set the step size per-iteration of SGD. Default 1.0. @@ -46,6 +70,14 @@ class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regPa this } + + + + + + + + /** * Set fraction of data to be used for each SGD iteration. Default 1.0. */ @@ -63,31 +95,77 @@ class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regPa } def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val initialWeights = Array.fill(nfeatures)(1.0) + train(input, initialWeights) + } + + def train( + input: RDD[(Double, Array[Double])], + initialWeights: Array[Double]): LogisticRegressionModel = { + // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => (y, Array(1.0, features:_*)) } - val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, regParam, miniBatchFraction) + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( + data, + new LogisticGradient(), + new SimpleUpdater(), + stepSize, + numIters, + 0.0, + initalWeightsWithIntercept, + miniBatchFraction) - val model = new LogisticRegressionModel(weightsScaled, intercept, losses) + val intercept = weights(0) + val weightsScaled = weights.tail - logInfo("Final model weights " + model.weights) + val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) + + logInfo("Final model weights " + model.weights.mkString(",")) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochastic losses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling Logistic Regression. + * NOTE(shivaram): We use multiple train methods instead of default arguments to support + * Java programs. */ object LogisticRegression_LocalRandomSGD { + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LogisticRegressionModel = + { + new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses @@ -96,18 +174,18 @@ object LogisticRegression_LocalRandomSGD { * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, stepSize: Double, - regParam: Double, + miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input) } /** @@ -117,18 +195,18 @@ object LogisticRegression_LocalRandomSGD { * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, - stepSize: Double, - regParam: Double) + stepSize: Double + ) : LogisticRegressionModel = { - train(input, numIterations, stepSize, regParam, 1.0) + train(input, numIterations, stepSize, 1.0) } /** @@ -145,7 +223,7 @@ object LogisticRegression_LocalRandomSGD { numIterations: Int) : LogisticRegressionModel = { - train(input, numIterations, 1.0, 0.0, 1.0) + train(input, numIterations, 1.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index bf10493bf5..2cd1d668eb 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.classification import scala.math.signum @@ -11,22 +28,31 @@ import org.jblas.DoubleMatrix * SVM using Stochastic Gradient Descent. */ class SVMModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, - val losses: Array[Double]) extends ClassificationModel { + val stochasticLosses: Array[Double]) extends ClassificationModel { + + // Create a column vector that can be used for predictions + private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) override def predict(testData: spark.RDD[Array[Double]]) = { + // A small optimization to avoid serializing the entire model. Only the weightsMatrix + // and intercept is needed. + val localWeights = weightsMatrix + val localIntercept = intercept testData.map { x => - signum(new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept) + signum(new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept) } } override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - signum(dataMat.dot(this.weights) + this.intercept) + signum(dataMat.dot(weightsMatrix) + this.intercept) } } + + class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -69,31 +95,77 @@ class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, va } def train(input: RDD[(Double, Array[Double])]): SVMModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val initialWeights = Array.fill(nfeatures)(1.0) + train(input, initialWeights) + } + + def train( + input: RDD[(Double, Array[Double])], + initialWeights: Array[Double]): SVMModel = { + // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => (y, Array(1.0, features:_*)) } - val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new HingeGradient(), new SquaredL2Updater(), stepSize, numIters, regParam, miniBatchFraction) + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( + data, + new HingeGradient(), + new SquaredL2Updater(), + stepSize, + numIters, + regParam, + initalWeightsWithIntercept, + miniBatchFraction) - val model = new SVMModel(weightsScaled, intercept, losses) + val intercept = weights(0) + val weightsScaled = weights.tail - logInfo("Final model weights " + model.weights) + val model = new SVMModel(weightsScaled, intercept, stochasticLosses) + + logInfo("Final model weights " + model.weights.mkString(",")) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochasticLosses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling SVM. + + */ object SVM_LocalRandomSGD { + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : SVMModel = + { + new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses @@ -151,7 +223,7 @@ object SVM_LocalRandomSGD { numIterations: Int) : SVMModel = { - train(input, numIterations, 1.0, 0.10, 1.0) + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 6ffc3b128b..4864ab7ccf 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import org.jblas.DoubleMatrix @@ -58,4 +75,3 @@ class HingeGradient extends Gradient { (DoubleMatrix.zeros(1,weights.length), 0.0) } } - diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index bd8489c386..8387d4939b 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import spark.{Logging, RDD, SparkContext} @@ -23,8 +40,9 @@ object GradientDescent { * @param miniBatchFraction - fraction of the input data set that should be used for * one iteration of SGD. Default value 1.0. * - * @return weights - Column matrix containing weights for every feature. - * @return lossHistory - Array containing the loss computed for every iteration. + * @return A tuple containing two elements. The first element is a column matrix containing + * weights for every feature, and the second element is an array containing the stochastic + * loss computed for every iteration. */ def runMiniBatchSGD( data: RDD[(Double, Array[Double])], @@ -33,16 +51,16 @@ object GradientDescent { stepSize: Double, numIters: Int, regParam: Double, - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + initialWeights: Array[Double], + miniBatchFraction: Double=1.0) : (Array[Double], Array[Double]) = { - val lossHistory = new ArrayBuffer[Double](numIters) + val stochasticLossHistory = new ArrayBuffer[Double](numIters) - val nfeatures: Int = data.take(1)(0)._2.length val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction - // Initialize weights as a column matrix - var weights = DoubleMatrix.ones(nfeatures) + // Initialize weights as a column vector + var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) var reg_val = 0.0 for (i <- 1 to numIters) { @@ -53,16 +71,17 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) + stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) weights = update._1 reg_val = update._2 - lossHistory.append(lossSum / miniBatchSize + reg_val) + stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) /*** Xinghao: The loss here is sum of lossSum computed using the weights before applying updater, and reg_val using weights after applying updater ***/ } - (weights, lossHistory.toArray) + (weights.toArray, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index bb2305c811..64364323a2 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import spark.{Logging, RDD, SparkContext} @@ -8,24 +25,34 @@ import org.jblas.DoubleMatrix /** * Lasso using Stochastic Gradient Descent. + * */ class LassoModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, - val losses: Array[Double]) extends RegressionModel { + val stochasticLosses: Array[Double]) extends RegressionModel { + + // Create a column vector that can be used for predictions + private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) override def predict(testData: spark.RDD[Array[Double]]) = { + // A small optimization to avoid serializing the entire model. Only the weightsMatrix + // and intercept is needed. + val localWeights = weightsMatrix + val localIntercept = intercept testData.map { x => - new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept + new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept } } + override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - dataMat.dot(this.weights) + this.intercept + dataMat.dot(weightsMatrix) + this.intercept } } + class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -68,31 +95,77 @@ class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, } def train(input: RDD[(Double, Array[Double])]): LassoModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val initialWeights = Array.fill(nfeatures)(1.0) + train(input, initialWeights) + } + + def train( + input: RDD[(Double, Array[Double])], + initialWeights: Array[Double]): LassoModel = { + // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => (y, Array(1.0, features:_*)) } - val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new SquaredGradient(), new L1Updater(), stepSize, numIters, regParam, miniBatchFraction) + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( + data, + new SquaredGradient(), + new L1Updater(), + stepSize, + numIters, + regParam, + initalWeightsWithIntercept, + miniBatchFraction) - val model = new LassoModel(weightsScaled, intercept, losses) + val intercept = weights(0) + val weightsScaled = weights.tail - logInfo("Final model weights " + model.weights) + val model = new LassoModel(weightsScaled, intercept, stochasticLosses) + + logInfo("Final model weights " + model.weights.mkString(",")) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochasticLosses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling Lasso. + * + * */ object Lasso_LocalRandomSGD { + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LassoModel = + { + new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses @@ -150,7 +223,7 @@ object Lasso_LocalRandomSGD { numIterations: Int) : LassoModel = { - train(input, numIterations, 1.0, 0.10, 1.0) + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 5741906a14..827ca66330 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -1,4 +1,25 @@ +<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala package spark.mllib.classification +======= +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.regression +>>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala import scala.util.Random @@ -6,7 +27,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { @@ -17,16 +37,23 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } +<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("LogisticRegression_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) +======= + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) +>>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val A = 2.0 - val B = -1.5 - // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) val unifRand = new scala.util.Random(45) val rLogis = (0 until nPoints).map { i => @@ -34,24 +61,87 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { math.log(u) - math.log(1.0-u) } - // y <- A + B*x + rlogis(100) + // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => - val yVal = A + B * x1(i) + rLogis(i) + val y: Seq[Double] = (0 until nPoints).map { i => + val yVal = offset + scale * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + val testData = (0 until nPoints).map(i => (y(i), Array(x1(i)))) + testData + } + + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() +<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala val lr = new LogisticRegression_LocalRandomSGD().setStepSize(10.0) .setNumIterations(20) +======= + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(20) +>>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala val model = lr.train(testRDD) - val weight0 = model.weights.get(0) + // Test the weights + val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) + } + + test("logistic regression with initial weights") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints, 42) + + val initialB = -1.0 + val initialWeights = Array(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(10) + + val model = lr.train(testRDD, initialWeights) + + val weight0 = model.weights(0) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } From 1714693324b5a3169d2e10dc16a2598f8e14d8b1 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 12:24:41 -0700 Subject: [PATCH 177/221] Current time called once with value now --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 266f180a50..5a18559ae9 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -34,6 +34,7 @@ import spark.executor.TaskMetrics private[spark] class StagePage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt + val now = System.currentTimeMillis() def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt @@ -54,7 +55,7 @@ private[spark] class StagePage(parent: JobProgressUI) { var activeTime = 0L listener.stageToTasksActive(stageId).foreach { t => - activeTime += t.timeRunning(System.currentTimeMillis()) + activeTime += t.timeRunning(now) } val summary = From dcc4743a950c7a9b767d7157ca62f512917ee6ae Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 12:52:53 -0700 Subject: [PATCH 178/221] Moved val now to render --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 5a18559ae9..ef72f42e30 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -34,10 +34,10 @@ import spark.executor.TaskMetrics private[spark] class StagePage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt - val now = System.currentTimeMillis() def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt + val now = System.currentTimeMillis() if (!listener.stageToTaskInfos.contains(stageId)) { val content = From 5a93e3c58c69574eaac6458f8515579b5bd03fd9 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 15:55:26 -0700 Subject: [PATCH 179/221] Cleaned up code based on pwendell's suggestions --- .../scala/spark/ui/exec/ExecutorsUI.scala | 12 ++---- .../main/scala/spark/ui/jobs/IndexPage.scala | 18 ++++---- .../scala/spark/ui/jobs/JobProgressUI.scala | 43 ++++--------------- 3 files changed, 20 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 80d00c6873..948b3017db 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -121,10 +121,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) { - executorToTasksActive(eid) = HashSet[TaskInfo]() - } - executorToTasksActive(eid) += taskStart.taskInfo + val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) + activeTasks += taskStart.taskInfo val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -133,10 +131,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) { - executorToTasksActive(eid) = HashSet[TaskInfo]() - } - executorToTasksActive(eid) -= taskEnd.taskInfo + val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) + activeTasks -= taskStart.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index b862c3539c..7e504c5f9f 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -125,16 +125,14 @@ private[spark] class IndexPage(parent: JobProgressUI) { case None => "Unknown" } - val shuffleRead = - if (!listener.hasShuffleRead(s.id)) - "" - else - Utils.memoryBytesToString(listener.stageToShuffleRead(s.id)) - val shuffleWrite = - if (!listener.hasShuffleWrite(s.id)) - "" - else - Utils.memoryBytesToString(listener.stageToShuffleWrite(s.id)) + val shuffleRead = listener.stageToShuffleRead(s.id) match { + case 0 => "" + case b => Utils.memoryBytesToString(b) + } + val shuffleWrite = listener.stageToShuffleWrite(s.id) match { + case 0 => "" + case b => Utils.memoryBytesToString(b) + } val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index e7fbff7f73..09d24b6302 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -65,6 +65,7 @@ private[spark] class JobProgressListener extends SparkListener { val completedStages = ListBuffer[Stage]() val failedStages = ListBuffer[Stage]() + // Total metrics reflect metrics only for completed tasks var totalTime = 0L var totalShuffleRead = 0L var totalShuffleWrite = 0L @@ -109,10 +110,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { val sid = taskStart.task.stageId - if (!stageToTasksActive.contains(sid)) { - stageToTasksActive(sid) = HashSet[TaskInfo]() - } - stageToTasksActive(sid) += taskStart.taskInfo + val tasksActive = stageToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) + tasksActive += taskStart.taskInfo val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -121,10 +120,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - if (!stageToTasksActive.contains(sid)) { - stageToTasksActive(sid) = HashSet[TaskInfo]() - } - stageToTasksActive(sid) -= taskEnd.taskInfo + val tasksActive = stageToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) + tasksActive -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -135,24 +132,18 @@ private[spark] class JobProgressListener extends SparkListener { (None, Option(taskEnd.taskMetrics)) } - if (!stageToTime.contains(sid)) { - stageToTime(sid) = 0L - } + stageToTime.getOrElseUpdate(sid, 0L) val time = metrics.map(m => m.executorRunTime).getOrElse(0) stageToTime(sid) += time totalTime += time - if (!stageToShuffleRead.contains(sid)) { - stageToShuffleRead(sid) = 0L - } + stageToShuffleRead.getOrElseUpdate(sid, 0L) val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead).getOrElse(0L) stageToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead - if (!stageToShuffleWrite.contains(sid)) { - stageToShuffleWrite(sid) = 0L - } + stageToShuffleWrite.getOrElseUpdate(sid, 0L) val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleBytesWritten).getOrElse(0L) stageToShuffleWrite(sid) += shuffleWrite @@ -178,22 +169,4 @@ private[spark] class JobProgressListener extends SparkListener { case _ => } } - - /** Is this stage's input from a shuffle read. */ - def hasShuffleRead(stageID: Int): Boolean = { - // This is written in a slightly complicated way to avoid having to scan all tasks - for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.flatMap(m => m.shuffleReadMetrics).isDefined - } - return false // No tasks have finished for this stage - } - - /** Is this stage's output to a shuffle write. */ - def hasShuffleWrite(stageID: Int): Boolean = { - // This is written in a slightly complicated way to avoid having to scan all tasks - for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.flatMap(m => m.shuffleWriteMetrics).isDefined - } - return false // No tasks have finished for this stage - } } From c2223e68012a44e9c4bebd97dc1db772df8c597e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 15 Jul 2013 08:48:53 -0700 Subject: [PATCH 180/221] Improve catch scope and logging for client stop() This does two things: 1. Catches the more general `TimeoutException`, since those can be thrown. 2. Logs at info level when a timeout is detected. --- core/src/main/scala/spark/deploy/client/Client.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index edefa0292d..6b038f5874 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -22,7 +22,7 @@ import akka.actor._ import akka.pattern.ask import akka.util.Duration import akka.util.duration._ -import akka.pattern.AskTimeoutException +import java.util.concurrent.TimeoutException import spark.{SparkException, Logging} import akka.remote.RemoteClientLifeCycleEvent import akka.remote.RemoteClientShutdown @@ -134,7 +134,8 @@ private[spark] class Client( val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { - case e: AskTimeoutException => // Ignore it, maybe master went away + case e: TimeoutException => + logInfo("Close request to Master timed out; it may already be shut down.") } actor = null } From 8177165ac41c1f39a09fa2a0b195f03928472fe5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 27 Jul 2013 16:01:54 -0700 Subject: [PATCH 181/221] Log executor on finish --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 860a38e9f8..bbf234febd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -564,8 +564,8 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: decreaseRunningTasks(1) if (!finished(index)) { tasksFinished += 1 - logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( - tid, info.duration, tasksFinished, numTasks)) + logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format( + tid, info.duration, info.hostPort, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler try { val result = ser.deserialize[TaskResult[_]](serializedData) From bcafb36c1ebe3d8f377ba69952235ae0a829bff8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 27 Jul 2013 16:03:50 -0700 Subject: [PATCH 182/221] Slight wording change --- core/src/main/scala/spark/deploy/client/Client.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 6b038f5874..29e494f495 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -135,7 +135,7 @@ private[spark] class Client( Await.result(future, timeout) } catch { case e: TimeoutException => - logInfo("Close request to Master timed out; it may already be shut down.") + logInfo("Stop request to Master timed out; it may already be shut down.") } actor = null } From 077f2dad226b4f817cb50ad5c07702e78506a698 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 16:39:36 -0700 Subject: [PATCH 183/221] Fixed outdated bugs --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/jobs/StagePage.scala | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 948b3017db..b70153fd30 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -132,7 +132,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) - activeTasks -= taskStart.taskInfo + activeTasks -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 7e504c5f9f..c12a4fe2a5 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -125,11 +125,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { case None => "Unknown" } - val shuffleRead = listener.stageToShuffleRead(s.id) match { + val shuffleRead = listener.stageToShuffleRead.getOrElse(s.id, 0L) match { case 0 => "" case b => Utils.memoryBytesToString(b) } - val shuffleWrite = listener.stageToShuffleWrite(s.id) match { + val shuffleWrite = listener.stageToShuffleWrite.getOrElse(s.id, 0L) match { case 0 => "" case b => Utils.memoryBytesToString(b) } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index ef72f42e30..e327cb3947 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -50,8 +50,8 @@ private[spark] class StagePage(parent: JobProgressUI) { val tasks = listener.stageToTaskInfos(stageId) - val shuffleRead = listener.hasShuffleRead(stageId) - val shuffleWrite = listener.hasShuffleWrite(stageId) + val shuffleRead = listener.stageToShuffleRead(stageId) > 0 + val shuffleWrite = listener.stageToShuffleWrite(stageId) > 0 var activeTime = 0L listener.stageToTasksActive(stageId).foreach { t => @@ -65,13 +65,13 @@ private[spark] class StagePage(parent: JobProgressUI) { CPU time: {parent.formatDuration(listener.stageToTime(stageId) + activeTime)} - {if (listener.hasShuffleRead(stageId)) + {if (shuffleRead)
  • Shuffle read: {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))}
  • } - {if (listener.hasShuffleWrite(stageId)) + {if (shuffleWrite)
  • Shuffle write: {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} From f11ad72d4ee2c6821749e1bf95c46d3f2c2cd860 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 27 Jul 2013 21:11:28 -0400 Subject: [PATCH 184/221] Some fixes to Python examples (style and package name for LR) --- python/examples/als.py | 5 ++--- python/examples/kmeans.py | 3 +-- python/examples/logistic_regression.py | 5 ++--- python/examples/pi.py | 3 +-- python/examples/transitive_closure.py | 5 ++--- python/examples/wordcount.py | 3 +-- 6 files changed, 9 insertions(+), 15 deletions(-) mode change 100644 => 100755 python/examples/kmeans.py mode change 100644 => 100755 python/examples/pi.py mode change 100644 => 100755 python/examples/transitive_closure.py mode change 100644 => 100755 python/examples/wordcount.py diff --git a/python/examples/als.py b/python/examples/als.py index f2b2eee64c..a77dfb2577 100755 --- a/python/examples/als.py +++ b/python/examples/als.py @@ -48,8 +48,7 @@ def update(i, vec, mat, ratings): if __name__ == "__main__": if len(sys.argv) < 2: - print >> sys.stderr, \ - "Usage: PythonALS " + print >> sys.stderr, "Usage: als " exit(-1) sc = SparkContext(sys.argv[1], "PythonALS", pyFiles=[realpath(__file__)]) M = int(sys.argv[2]) if len(sys.argv) > 2 else 100 @@ -84,5 +83,5 @@ if __name__ == "__main__": usb = sc.broadcast(us) error = rmse(R, ms, us) - print "Iteration %d:" % i + print "Iteration %d:" % i print "\nRMSE: %5.4f\n" % error diff --git a/python/examples/kmeans.py b/python/examples/kmeans.py old mode 100644 new mode 100755 index c670556f2b..ba31af92fc --- a/python/examples/kmeans.py +++ b/python/examples/kmeans.py @@ -41,8 +41,7 @@ def closestPoint(p, centers): if __name__ == "__main__": if len(sys.argv) < 5: - print >> sys.stderr, \ - "Usage: PythonKMeans " + print >> sys.stderr, "Usage: kmeans " exit(-1) sc = SparkContext(sys.argv[1], "PythonKMeans") lines = sc.textFile(sys.argv[2]) diff --git a/python/examples/logistic_regression.py b/python/examples/logistic_regression.py index 54d227d0d3..3ac1bae4e9 100755 --- a/python/examples/logistic_regression.py +++ b/python/examples/logistic_regression.py @@ -35,7 +35,7 @@ np.random.seed(42) DataPoint = namedtuple("DataPoint", ['x', 'y']) -from lr import DataPoint # So that DataPoint is properly serialized +from logistic_regression import DataPoint # So that DataPoint is properly serialized def generateData(): @@ -48,8 +48,7 @@ def generateData(): if __name__ == "__main__": if len(sys.argv) == 1: - print >> sys.stderr, \ - "Usage: PythonLR []" + print >> sys.stderr, "Usage: logistic_regression []" exit(-1) sc = SparkContext(sys.argv[1], "PythonLR", pyFiles=[realpath(__file__)]) slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 diff --git a/python/examples/pi.py b/python/examples/pi.py old mode 100644 new mode 100755 index 33c026e824..ab0645fc2f --- a/python/examples/pi.py +++ b/python/examples/pi.py @@ -24,8 +24,7 @@ from pyspark import SparkContext if __name__ == "__main__": if len(sys.argv) == 1: - print >> sys.stderr, \ - "Usage: PythonPi []" + print >> sys.stderr, "Usage: pi []" exit(-1) sc = SparkContext(sys.argv[1], "PythonPi") slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 diff --git a/python/examples/transitive_closure.py b/python/examples/transitive_closure.py old mode 100644 new mode 100755 index 40be3b5000..744cce6651 --- a/python/examples/transitive_closure.py +++ b/python/examples/transitive_closure.py @@ -37,10 +37,9 @@ def generateGraph(): if __name__ == "__main__": if len(sys.argv) == 1: - print >> sys.stderr, \ - "Usage: PythonTC []" + print >> sys.stderr, "Usage: transitive_closure []" exit(-1) - sc = SparkContext(sys.argv[1], "PythonTC") + sc = SparkContext(sys.argv[1], "PythonTransitiveClosure") slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 tc = sc.parallelize(generateGraph(), slices).cache() diff --git a/python/examples/wordcount.py b/python/examples/wordcount.py old mode 100644 new mode 100755 index 41c846ba79..a6de22766a --- a/python/examples/wordcount.py +++ b/python/examples/wordcount.py @@ -23,8 +23,7 @@ from pyspark import SparkContext if __name__ == "__main__": if len(sys.argv) < 3: - print >> sys.stderr, \ - "Usage: PythonWordCount " + print >> sys.stderr, "Usage: wordcount " exit(-1) sc = SparkContext(sys.argv[1], "PythonWordCount") lines = sc.textFile(sys.argv[2], 1) From ccfa362ddec1bc942785798dea41c3aac52df60f Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 10:33:57 -0700 Subject: [PATCH 185/221] Change *_LocalRandomSGD to *LocalRandomSGD --- .../classification/LogisticRegression.scala | 10 ++++---- .../spark/mllib/classification/SVM.scala | 10 ++++---- .../scala/spark/mllib/regression/Lasso.scala | 10 ++++---- .../LogisticRegressionSuite.scala | 23 +++---------------- .../spark/mllib/classification/SVMSuite.scala | 4 ++-- .../spark/mllib/regression/LassoSuite.scala | 8 +++---- 6 files changed, 24 insertions(+), 41 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 40b96fbe3a..1b093187f2 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -53,7 +53,7 @@ class LogisticRegressionModel( } } -class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var miniBatchFraction: Double, +class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -138,7 +138,7 @@ class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var miniB * NOTE(shivaram): We use multiple train methods instead of default arguments to support * Java programs. */ -object LogisticRegression_LocalRandomSGD { +object LogisticRegressionLocalRandomSGD { /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number @@ -163,7 +163,7 @@ object LogisticRegression_LocalRandomSGD { initialWeights: Array[Double]) : LogisticRegressionModel = { - new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) + new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) } /** @@ -185,7 +185,7 @@ object LogisticRegression_LocalRandomSGD { miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input) + new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input) } /** @@ -233,7 +233,7 @@ object LogisticRegression_LocalRandomSGD { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegression_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LogisticRegressionLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 2cd1d668eb..76844f6b9c 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -53,7 +53,7 @@ class SVMModel( -class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -138,7 +138,7 @@ class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, va */ -object SVM_LocalRandomSGD { +object SVMLocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number @@ -163,7 +163,7 @@ object SVM_LocalRandomSGD { initialWeights: Array[Double]) : SVMModel = { - new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) } /** @@ -185,7 +185,7 @@ object SVM_LocalRandomSGD { miniBatchFraction: Double) : SVMModel = { - new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -233,7 +233,7 @@ object SVM_LocalRandomSGD { } val sc = new SparkContext(args(0), "SVM") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = SVM_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = SVMLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index 64364323a2..1952658bb2 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -53,7 +53,7 @@ class LassoModel( } -class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class LassoLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -138,7 +138,7 @@ class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, * * */ -object Lasso_LocalRandomSGD { +object LassoLocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number @@ -163,7 +163,7 @@ object Lasso_LocalRandomSGD { initialWeights: Array[Double]) : LassoModel = { - new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) } /** @@ -185,7 +185,7 @@ object Lasso_LocalRandomSGD { miniBatchFraction: Double) : LassoModel = { - new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -233,7 +233,7 @@ object Lasso_LocalRandomSGD { } val sc = new SparkContext(args(0), "Lasso") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = Lasso_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LassoLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 827ca66330..144b8b1bc7 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -1,6 +1,3 @@ -<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala -package spark.mllib.classification -======= /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -18,8 +15,7 @@ package spark.mllib.classification * limitations under the License. */ -package spark.mllib.regression ->>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +package spark.mllib.classification import scala.util.Random @@ -37,13 +33,6 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } -<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("LogisticRegression_LocalRandomSGD") { - val nPoints = 10000 - val rnd = new Random(42) - -======= // Generate input of the form Y = logistic(offset + scale*X) def generateLogisticInput( offset: Double, @@ -51,7 +40,6 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { nPoints: Int, seed: Int): Seq[(Double, Array[Double])] = { val rnd = new Random(seed) ->>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) @@ -91,12 +79,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() -<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala - val lr = new LogisticRegression_LocalRandomSGD().setStepSize(10.0) - .setNumIterations(20) -======= - val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(20) ->>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala + val lr = new LogisticRegressionLocalRandomSGD().setStepSize(10.0).setNumIterations(20) val model = lr.train(testRDD) @@ -128,7 +111,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { testRDD.cache() // Use half as many iterations as the previous test. - val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(10) + val lr = new LogisticRegressionLocalRandomSGD().setStepSize(10.0).setNumIterations(10) val model = lr.train(testRDD, initialWeights) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 50cf260f49..0d781c310c 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -19,7 +19,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("SVM_LocalRandomSGD") { + test("SVMLocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -46,7 +46,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { writer_data.write("\n")}) writer_data.close() - val svm = new SVM_LocalRandomSGD().setStepSize(1.0) + val svm = new SVMLocalRandomSGD().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100) diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 9836ac54c1..0c39e1e09b 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -17,7 +17,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("Lasso_LocalRandomSGD") { + test("LassoLocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -36,14 +36,14 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val ls = new Lasso_LocalRandomSGD().setStepSize(1.0) + val ls = new LassoLocalRandomSGD().setStepSize(1.0) .setRegParam(0.01) .setNumIterations(20) val model = ls.train(testRDD) - val weight0 = model.weights.get(0) - val weight1 = model.weights.get(1) + val weight0 = model.weights(0) + val weight1 = model.weights(1) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") From 72ff62a37c7310bab02f0231e91d3ba4d423217a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 22:21:04 -0400 Subject: [PATCH 186/221] Two fixes to IPython support: - Don't attempt to run worker processes with ipython (that can cause some crashes as ipython prints things to standard out) - Allow passing some IPYTHON_OPTS to launch things like the notebook --- pyspark | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/pyspark b/pyspark index 37a355462e..801239c108 100755 --- a/pyspark +++ b/pyspark @@ -53,9 +53,13 @@ if [[ "$SPARK_LAUNCH_WITH_SCALA" != "0" ]] ; then export SPARK_LAUNCH_WITH_SCALA=1 fi -if [[ "$IPYTHON" = "1" ]] ; then - export PYSPARK_PYTHON="ipython" - exec "$PYSPARK_PYTHON" -i -c "%run $PYTHONSTARTUP" -else - exec "$PYSPARK_PYTHON" "$@" +if [ -n "$IPYTHON_OPTS" ]; then + IPYTHON=1 +fi + +if [[ "$IPYTHON" = "1" ]] ; then + IPYTHON_OPTS=${IPYTHON_OPTS:--i} + exec ipython "$IPYTHON_OPTS" -c "%run $PYTHONSTARTUP" +else + exec "$PYSPARK_PYTHON" "$@" fi From 29e042940ac79e42e2f8818ceda6a962a76948ac Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 20:39:52 -0700 Subject: [PATCH 187/221] Move data generators to util --- .../LassoGenerator.scala => util/LassoDataGenerator.scala} | 0 .../SVMGenerator.scala => util/SVMDataGenerator.scala} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename mllib/src/main/scala/spark/mllib/{regression/LassoGenerator.scala => util/LassoDataGenerator.scala} (100%) rename mllib/src/main/scala/spark/mllib/{classification/SVMGenerator.scala => util/SVMDataGenerator.scala} (100%) diff --git a/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala From 67de051bbb81096dc37ea6f92a82a9224b4af61e Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 21:09:56 -0700 Subject: [PATCH 188/221] SVMSuite and LassoSuite rewritten to follow closely with LogisticRegressionSuite --- .../spark/mllib/classification/SVMSuite.scala | 101 ++++++++++++++---- .../spark/mllib/regression/LassoSuite.scala | 97 ++++++++++++++--- 2 files changed, 162 insertions(+), 36 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 0d781c310c..2a23825acc 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.classification import scala.util.Random @@ -7,7 +24,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ import java.io._ @@ -19,43 +35,82 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } + // Generate noisy input of the form Y = signum(x.dot(weights) + intercept + noise) + def generateSVMInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) + val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) + val y = x.map(xi => + signum((xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + ) + y zip x + } + + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + test("SVMLocalRandomSGD") { val nPoints = 10000 - val rnd = new Random(42) - - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) val A = 2.0 val B = -1.5 val C = 1.0 - val y = (0 until nPoints).map { i => - signum(A + B * x1(i) + C * x2(i) + 0.0*rnd.nextGaussian()) - } - - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + val testData = generateSVMInput(A, Array[Double](B,C), nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val writer_data = new PrintWriter(new File("svmtest.dat")) - testData.foreach(yx => { - writer_data.write(yx._1 + "") - yx._2.foreach(xi => writer_data.write("\t" + xi)) - writer_data.write("\n")}) - writer_data.close() - - val svm = new SVMLocalRandomSGD().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100) + val svm = new SVMLocalRandomSGD().setStepSize(1.0).setRegParam(1.0).setNumIterations(100) val model = svm.train(testRDD) - val yPredict = (0 until nPoints).map(i => model.predict(Array(x1(i),x2(i)))) + val validationData = generateSVMInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) - val accuracy = ((y zip yPredict).map(yy => if (yy._1==yy._2) 1 else 0).reduceLeft(_+_).toDouble / nPoints.toDouble) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) - assert(accuracy >= 0.90, "Accuracy (" + accuracy + ") too low") + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) + } + + test("SVMLocalRandomSGD with initial weights") { + val nPoints = 10000 + + val A = 2.0 + val B = -1.5 + val C = 1.0 + + val testData = generateSVMInput(A, Array[Double](B,C), nPoints, 42) + + val initialB = -1.0 + val initialC = -1.0 + val initialWeights = Array(initialB,initialC) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val svm = new SVMLocalRandomSGD().setStepSize(1.0).setRegParam(1.0).setNumIterations(100) + + val model = svm.train(testRDD, initialWeights) + + val validationData = generateSVMInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 0c39e1e09b..33e87dfd9f 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random @@ -6,7 +23,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ class LassoSuite extends FunSuite with BeforeAndAfterAll { @@ -17,35 +33,90 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } + // Generate noisy input of the form Y = x.dot(weights) + intercept + noise + def generateLassoInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) + val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) + val y = x.map(xi => (xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + y zip x + } + + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + test("LassoLocalRandomSGD") { val nPoints = 10000 - val rnd = new Random(42) - - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) val A = 2.0 val B = -1.5 val C = 1.0e-2 - val y = (0 until nPoints).map { i => - A + B * x1(i) + C * x2(i) + 0.1*rnd.nextGaussian() - } - - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + val testData = generateLassoInput(A, Array[Double](B,C), nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val ls = new LassoLocalRandomSGD().setStepSize(1.0) - .setRegParam(0.01) - .setNumIterations(20) + val ls = new LassoLocalRandomSGD().setStepSize(1.0).setRegParam(0.01).setNumIterations(20) val model = ls.train(testRDD) val weight0 = model.weights(0) val weight1 = model.weights(1) + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + + val validationData = generateLassoInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) + } + + test("LassoLocalRandomSGD with initial weights") { + val nPoints = 10000 + + val A = 2.0 + val B = -1.5 + val C = 1.0e-2 + + val testData = generateLassoInput(A, Array[Double](B,C), nPoints, 42) + + val initialB = -1.0 + val initialC = -1.0 + val initialWeights = Array(initialB,initialC) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val ls = new LassoLocalRandomSGD().setStepSize(1.0).setRegParam(0.01).setNumIterations(20) + + val model = ls.train(testRDD, initialWeights) + + val weight0 = model.weights(0) + val weight1 = model.weights(1) assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + + val validationData = generateLassoInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } From 9398dced0331c0ec098ef5eb4616571874ceefb6 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 21:39:19 -0700 Subject: [PATCH 189/221] Changed Classification to return Int instead of Double Also minor changes to formatting and comments --- .../mllib/classification/Classification.scala | 8 ++++---- .../classification/LogisticRegression.scala | 16 ++++------------ .../scala/spark/mllib/classification/SVM.scala | 8 ++++---- .../spark/mllib/optimization/Gradient.scala | 4 ++-- .../mllib/optimization/GradientDescent.scala | 8 ++++---- .../scala/spark/mllib/optimization/Updater.scala | 12 ++++++++++-- .../spark/mllib/classification/SVMSuite.scala | 2 -- 7 files changed, 28 insertions(+), 30 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/Classification.scala index 96d7a54f18..d6154b66ae 100644 --- a/mllib/src/main/scala/spark/mllib/classification/Classification.scala +++ b/mllib/src/main/scala/spark/mllib/classification/Classification.scala @@ -7,15 +7,15 @@ trait ClassificationModel extends Serializable { * Predict values for the given data set using the model trained. * * @param testData RDD representing data points to be predicted - * @return RDD[Double] where each entry contains the corresponding prediction + * @return RDD[Int] where each entry contains the corresponding prediction */ - def predict(testData: RDD[Array[Double]]): RDD[Double] + def predict(testData: RDD[Array[Double]]): RDD[Int] /** * Predict values for a single data point using the model trained. * * @param testData array representing a single data point - * @return Double prediction from the trained model + * @return Int prediction from the trained model */ - def predict(testData: Array[Double]): Double + def predict(testData: Array[Double]): Int } diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 1b093187f2..0a7effb1d7 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -35,21 +35,21 @@ class LogisticRegressionModel( // Create a column vector that can be used for predictions private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) - override def predict(testData: spark.RDD[Array[Double]]) = { + override def predict(testData: spark.RDD[Array[Double]]): RDD[Int] = { // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. val localWeights = weightsMatrix val localIntercept = intercept testData.map { x => val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept - 1.0/ (1.0 + math.exp(margin * -1)) + (1.0/ (1.0 + math.exp(margin * -1))).toInt } } - override def predict(testData: Array[Double]): Double = { + override def predict(testData: Array[Double]): Int = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept - 1.0/ (1.0 + math.exp(margin * -1)) + (1.0/ (1.0 + math.exp(margin * -1))).toInt } } @@ -70,14 +70,6 @@ class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBa this } - - - - - - - - /** * Set fraction of data to be used for each SGD iteration. Default 1.0. */ diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 76844f6b9c..30766a4c64 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -35,19 +35,19 @@ class SVMModel( // Create a column vector that can be used for predictions private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) - override def predict(testData: spark.RDD[Array[Double]]) = { + override def predict(testData: spark.RDD[Array[Double]]): RDD[Int] = { // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. val localWeights = weightsMatrix val localIntercept = intercept testData.map { x => - signum(new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept) + signum(new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept).toInt } } - override def predict(testData: Array[Double]): Double = { + override def predict(testData: Array[Double]): Int = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - signum(dataMat.dot(weightsMatrix) + this.intercept) + signum(dataMat.dot(weightsMatrix) + this.intercept).toInt } } diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 4864ab7ccf..22b2ec5ed6 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -70,8 +70,8 @@ class HingeGradient extends Gradient { val dotProduct = data.dot(weights) if (1.0 > label * dotProduct) - (data.mul(-label), 1.0 - label * dotProduct) + (data.mul(-label), 1.0 - label * dotProduct) else - (DoubleMatrix.zeros(1,weights.length), 0.0) + (DoubleMatrix.zeros(1,weights.length), 0.0) } } diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 8387d4939b..d4b83a1456 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -76,10 +76,10 @@ object GradientDescent { weights = update._1 reg_val = update._2 stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) - /*** - Xinghao: The loss here is sum of lossSum computed using the weights before applying updater, - and reg_val using weights after applying updater - ***/ + /* + * NOTE(Xinghao): The loss here is sum of lossSum computed using the weights before applying updater, + * and reg_val using weights after applying updater + */ } (weights.toArray, stochasticLossHistory.toArray) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index cd344a6680..188fe7d972 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -46,17 +46,25 @@ class SimpleUpdater extends Updater { } /** -L1 regularization -- corresponding proximal operator is the soft-thresholding function +* L1 regularization -- corresponding proximal operator is the soft-thresholding function +* That is, each weight component is shrunk towards 0 by shrinkageVal +* If w > shrinkageVal, set weight component to w-shrinkageVal. +* If w < -shrinkageVal, set weight component to w+shrinkageVal. +* If -shrinkageVal < w < shrinkageVal, set weight component to 0. +* Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) **/ class L1Updater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) val normGradient = gradient.mul(thisIterStepSize) + // Take gradient step val newWeights = weightsOld.sub(normGradient) + // Soft thresholding + val shrinkageVal = regParam * thisIterStepSize (0 until newWeights.length).foreach(i => { val wi = newWeights.get(i) - newWeights.put(i, signum(wi) * max(0.0, abs(wi) - regParam * thisIterStepSize)) + newWeights.put(i, signum(wi) * max(0.0, abs(wi) - shrinkageVal)) }) (newWeights, newWeights.norm1 * regParam) } diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 2a23825acc..91c037e9b1 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -25,8 +25,6 @@ import org.scalatest.FunSuite import spark.SparkContext -import java.io._ - class SVMSuite extends FunSuite with BeforeAndAfterAll { val sc = new SparkContext("local", "test") From 96e04f4cb7de3a7c9d31aa7acba496d81066634e Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 22:12:39 -0700 Subject: [PATCH 190/221] Fixed SVM and LR train functions to take Int instead of Double for Classification --- .../classification/LogisticRegression.scala | 17 ++++++++--------- .../scala/spark/mllib/classification/SVM.scala | 16 ++++++++-------- .../LogisticRegressionSuite.scala | 10 +++++----- 3 files changed, 21 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 0a7effb1d7..cbc0d03ae1 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -86,19 +86,19 @@ class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBa this } - def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + def train(input: RDD[(Int, Array[Double])]): LogisticRegressionModel = { val nfeatures: Int = input.take(1)(0)._2.length val initialWeights = Array.fill(nfeatures)(1.0) train(input, initialWeights) } def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], initialWeights: Array[Double]): LogisticRegressionModel = { // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => - (y, Array(1.0, features:_*)) + (y.toDouble, Array(1.0, features:_*)) } val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) @@ -141,13 +141,12 @@ object LogisticRegressionLocalRandomSGD { * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. * @param initialWeights Initial set of weights to be used. Array should be equal in size to * the number of features in the data. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, @@ -170,7 +169,7 @@ object LogisticRegressionLocalRandomSGD { * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, @@ -192,7 +191,7 @@ object LogisticRegressionLocalRandomSGD { * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double ) @@ -211,7 +210,7 @@ object LogisticRegressionLocalRandomSGD { * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int) : LogisticRegressionModel = { @@ -224,7 +223,7 @@ object LogisticRegressionLocalRandomSGD { System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)).map(yx => (yx._1.toInt, yx._2)) val model = LogisticRegressionLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 30766a4c64..15b689e7e0 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -94,19 +94,19 @@ class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, var this } - def train(input: RDD[(Double, Array[Double])]): SVMModel = { + def train(input: RDD[(Int, Array[Double])]): SVMModel = { val nfeatures: Int = input.take(1)(0)._2.length val initialWeights = Array.fill(nfeatures)(1.0) train(input, initialWeights) } def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], initialWeights: Array[Double]): SVMModel = { // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => - (y, Array(1.0, features:_*)) + (y.toDouble, Array(1.0, features:_*)) } val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) @@ -155,7 +155,7 @@ object SVMLocalRandomSGD { * the number of features in the data. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, regParam: Double, @@ -178,7 +178,7 @@ object SVMLocalRandomSGD { * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, regParam: Double, @@ -200,7 +200,7 @@ object SVMLocalRandomSGD { * @return a SVMModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, regParam: Double) @@ -219,7 +219,7 @@ object SVMLocalRandomSGD { * @return a SVMModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int) : SVMModel = { @@ -232,7 +232,7 @@ object SVMLocalRandomSGD { System.exit(1) } val sc = new SparkContext(args(0), "SVM") - val data = MLUtils.loadLabeledData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)).map(yx => (yx._1.toInt, yx._2)) val model = SVMLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 144b8b1bc7..3aa9fe6d12 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -38,7 +38,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { offset: Double, scale: Double, nPoints: Int, - seed: Int): Seq[(Double, Array[Double])] = { + seed: Int): Seq[(Int, Array[Double])] = { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) @@ -51,19 +51,19 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) - val y: Seq[Double] = (0 until nPoints).map { i => + val y: Seq[Int] = (0 until nPoints).map { i => val yVal = offset + scale * x1(i) + rLogis(i) - if (yVal > 0) 1.0 else 0.0 + if (yVal > 0) 1 else 0 } val testData = (0 until nPoints).map(i => (y(i), Array(x1(i)))) testData } - def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => // A prediction is off if the prediction is more than 0.5 away from expected value. - math.abs(prediction - expected) > 0.5 + math.abs(prediction.toDouble - expected.toDouble) > 0.5 }.size // At least 80% of the predictions should be on. assert(numOffPredictions < input.length / 5) From c823ee1e2bea7cde61cb4411a0f0db91f1df2af2 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 22:17:53 -0700 Subject: [PATCH 191/221] Replace map-reduce with dot operator using DoubleMatrix --- .../main/scala/spark/mllib/util/LassoDataGenerator.scala | 4 ++-- .../main/scala/spark/mllib/util/SVMDataGenerator.scala | 6 ++++-- .../test/scala/spark/mllib/classification/SVMSuite.scala | 9 ++++++--- .../test/scala/spark/mllib/regression/LassoSuite.scala | 7 ++++++- 4 files changed, 18 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala index d2d3bb33c7..1ff32d2e7f 100644 --- a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala @@ -26,7 +26,7 @@ object LassoGenerator { val sc = new SparkContext(sparkMaster, "LassoGenerator") val globalRnd = new Random(94720) - val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -34,7 +34,7 @@ object LassoGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val y = ((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1 + val y = (new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1 (y, x) } diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala index 029f262660..ced52093f5 100644 --- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala @@ -8,6 +8,8 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.util.MLUtils +import org.jblas.DoubleMatrix + object SVMGenerator { def main(args: Array[String]) { @@ -27,7 +29,7 @@ object SVMGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -35,7 +37,7 @@ object SVMGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val y = signum(((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1) + val y = signum((new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1) (y, x) } diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 91c037e9b1..3f00398a0a 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -25,6 +25,8 @@ import org.scalatest.FunSuite import spark.SparkContext +import org.jblas.DoubleMatrix + class SVMSuite extends FunSuite with BeforeAndAfterAll { val sc = new SparkContext("local", "test") @@ -38,16 +40,17 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { intercept: Double, weights: Array[Double], nPoints: Int, - seed: Int): Seq[(Double, Array[Double])] = { + seed: Int): Seq[(Int, Array[Double])] = { val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) val y = x.map(xi => - signum((xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + signum((new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian()).toInt ) y zip x } - def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => // A prediction is off if the prediction is more than 0.5 away from expected value. math.abs(prediction - expected) > 0.5 diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 33e87dfd9f..cf2b067d40 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -24,6 +24,8 @@ import org.scalatest.FunSuite import spark.SparkContext +import org.jblas.DoubleMatrix + class LassoSuite extends FunSuite with BeforeAndAfterAll { val sc = new SparkContext("local", "test") @@ -40,8 +42,11 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { nPoints: Int, seed: Int): Seq[(Double, Array[Double])] = { val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) - val y = x.map(xi => (xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + val y = x.map(xi => + (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian() + ) y zip x } From b9d6783f36d527f5082bf13a4ee6fd108e97795c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:28:42 -0400 Subject: [PATCH 192/221] Optimize Python take() to not compute entire first partition --- .../scala/spark/api/python/PythonRDD.scala | 64 +++++++++++-------- python/pyspark/rdd.py | 15 +++-- 2 files changed, 45 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index af10822dbd..2dd79f7100 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -63,34 +63,42 @@ private[spark] class PythonRDD[T: ClassManifest]( // Start a thread to feed the process input from our parent's iterator new Thread("stdin writer for " + pythonExec) { override def run() { - SparkEnv.set(env) - val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) - val dataOut = new DataOutputStream(stream) - val printOut = new PrintWriter(stream) - // Partition index - dataOut.writeInt(split.index) - // sparkFilesDir - PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) - // Broadcast variables - dataOut.writeInt(broadcastVars.length) - for (broadcast <- broadcastVars) { - dataOut.writeLong(broadcast.id) - dataOut.writeInt(broadcast.value.length) - dataOut.write(broadcast.value) + try { + SparkEnv.set(env) + val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) + val dataOut = new DataOutputStream(stream) + val printOut = new PrintWriter(stream) + // Partition index + dataOut.writeInt(split.index) + // sparkFilesDir + PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) + // Broadcast variables + dataOut.writeInt(broadcastVars.length) + for (broadcast <- broadcastVars) { + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) + } + dataOut.flush() + // Serialized user code + for (elem <- command) { + printOut.println(elem) + } + printOut.flush() + // Data values + for (elem <- parent.iterator(split, context)) { + PythonRDD.writeAsPickle(elem, dataOut) + } + dataOut.flush() + printOut.flush() + worker.shutdownOutput() + } catch { + 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) } - dataOut.flush() - // Serialized user code - for (elem <- command) { - printOut.println(elem) - } - printOut.flush() - // Data values - for (elem <- parent.iterator(split, context)) { - PythonRDD.writeAsPickle(elem, dataOut) - } - dataOut.flush() - printOut.flush() - worker.shutdownOutput() } }.start() @@ -297,7 +305,7 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) Utils.checkHost(serverHost, "Expected hostname") val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - + override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList override def addInPlace(val1: JList[Array[Byte]], val2: JList[Array[Byte]]) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c6a6b24c5a..6efa61aa66 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -386,13 +386,16 @@ class RDD(object): >>> sc.parallelize([2, 3, 4, 5, 6]).take(10) [2, 3, 4, 5, 6] """ + def takeUpToNum(iterator): + taken = 0 + while taken < num: + yield next(iterator) + taken += 1 + # Take only up to num elements from each partition we try + mapped = self.mapPartitions(takeUpToNum) items = [] - for partition in range(self._jrdd.splits().size()): - iterator = self.ctx._takePartition(self._jrdd.rdd(), partition) - # Each item in the iterator is a string, Python object, batch of - # Python objects. Regardless, it is sufficient to take `num` - # of these objects in order to collect `num` Python objects: - iterator = iterator.take(num) + for partition in range(mapped._jrdd.splits().size()): + iterator = self.ctx._takePartition(mapped._jrdd.rdd(), partition) items.extend(self._collect_iterator_through_file(iterator)) if len(items) >= num: break From b5ec35562210c8e7ca4fea07a0d46cb255988c0d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:38:56 -0400 Subject: [PATCH 193/221] Optimize Python foreach() to not return as many objects --- python/pyspark/rdd.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6efa61aa66..4aafe35d13 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -267,7 +267,11 @@ class RDD(object): >>> def f(x): print x >>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f) """ - self.map(f).collect() # Force evaluation + def processPartition(iterator): + for x in iterator: + f(x) + yield None + self.mapPartitions(processPartition).collect() # Force evaluation def collect(self): """ From 96b50e82dc0db501ab94748a23ae3237f1d82034 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:39:27 -0400 Subject: [PATCH 194/221] Allow python/run-tests to run from any directory --- python/run-tests | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/run-tests b/python/run-tests index 1ee947d414..6643faa2e0 100755 --- a/python/run-tests +++ b/python/run-tests @@ -21,6 +21,9 @@ # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; cd ../; pwd)" +# CD into the python directory to find things on the right path +cd "$FWDIR/python" + FAILED=0 $FWDIR/pyspark pyspark/rdd.py From d75c3086951f603ec30b2527c24559e053ed7f25 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:50:38 -0400 Subject: [PATCH 195/221] Use None instead of empty string as it's slightly smaller/faster --- python/pyspark/rdd.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 4aafe35d13..8734cacb0b 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -160,7 +160,7 @@ class RDD(object): >>> sorted(sc.parallelize([1, 1, 2, 3]).distinct().collect()) [1, 2, 3] """ - return self.map(lambda x: (x, "")) \ + return self.map(lambda x: (x, None)) \ .reduceByKey(lambda x, _: x) \ .map(lambda (x, _): x) From feba7ee540fca28872957120e5e39b9e36466953 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 00:09:11 -0400 Subject: [PATCH 196/221] SPARK-815. Python parallelize() should split lists before batching One unfortunate consequence of this fix is that we materialize any collections that are given to us as generators, but this seems necessary to get reasonable behavior on small collections. We could add a batchSize parameter later to bypass auto-computation of batch size if this becomes a problem (e.g. if users really want to parallelize big generators nicely) --- python/pyspark/context.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 2f741cb345..c2b49ff37a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -141,14 +141,21 @@ class SparkContext(object): def parallelize(self, c, numSlices=None): """ Distribute a local Python collection to form an RDD. + + >>> sc.parallelize(range(5), 5).glom().collect() + [[0], [1], [2], [3], [4]] """ numSlices = numSlices or self.defaultParallelism # Calling the Java parallelize() method with an ArrayList is too slow, # because it sends O(n) Py4J commands. As an alternative, serialized # objects are written to a file and loaded through textFile(). tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) - if self.batchSize != 1: - c = batched(c, self.batchSize) + # Make sure we distribute data evenly if it's smaller than self.batchSize + if "__len__" not in dir(c): + c = list(c) # Make it a list so we can compute its length + batchSize = min(len(c) // numSlices, self.batchSize) + if batchSize > 1: + c = batched(c, batchSize) for x in c: write_with_length(dump_pickle(x), tempFile) tempFile.close() From 497f55755f8374dd553f21cc3c5edcc6e3f5acce Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 02:50:48 -0400 Subject: [PATCH 197/221] Add docs about ipython --- docs/python-programming-guide.md | 34 +++++++++++++++++++++++++++++--- 1 file changed, 31 insertions(+), 3 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index e8aaac74d0..794bff5647 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -10,6 +10,7 @@ To learn the basics of Spark, we recommend reading through the easy to follow even if you don't know Scala. This guide will show how to use the Spark features described there in Python. + # Key Differences in the Python API There are a few key differences between the Python and Scala APIs: @@ -50,6 +51,7 @@ PySpark will automatically ship these functions to workers, along with any objec Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers. The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers. + # Installing and Configuring PySpark PySpark requires Python 2.6 or higher. @@ -81,16 +83,41 @@ The Python shell can be used explore data interactively and is a simple way to l >>> help(pyspark) # Show all pyspark functions {% endhighlight %} -By default, the `pyspark` shell creates SparkContext that runs jobs locally. -To connect to a non-local cluster, set the `MASTER` environment variable. +By default, the `pyspark` shell creates SparkContext that runs jobs locally on a single core. +To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable. For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html): {% highlight bash %} $ MASTER=spark://IP:PORT ./pyspark {% endhighlight %} +Or, to use four cores on the local machine: -# Standalone Use +{% highlight bash %} +$ MASTER=local[4] ./pyspark +{% endhighlight %} + + +## IPython + +It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter. +To do this, simply set the `IPYTHON` variable to `1` when running `pyspark`: + +{% highlight bash %} +$ IPYTHON=1 ./pyspark +{% endhighlight %} + +Alternatively, you can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch +the [IPython Notebook](http://ipython.org/notebook.html) with PyLab graphing support: + +{% highlight bash %} +$ IPYTHON_OPTS="notebook --pylab inline" ./pyspark +{% endhighlight %} + +IPython also works on a cluster or on multiple cores if you set the `MASTER` environment variable. + + +# Standalone Programs PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `pyspark`. The Quick Start guide includes a [complete example](quick-start.html#a-standalone-job-in-python) of a standalone Python job. @@ -105,6 +132,7 @@ sc = SparkContext("local", "Job Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines. Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. + # Where to Go from Here PySpark includes several sample programs in the [`python/examples` folder](https://github.com/mesos/spark/tree/master/python/examples). From 75f375730025788a5982146d97bf3df9ef69ab23 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:19:56 -0700 Subject: [PATCH 198/221] Fix rounding error in LogisticRegression.scala --- .../spark/mllib/classification/LogisticRegression.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index cbc0d03ae1..bc1c327729 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -21,6 +21,8 @@ import spark.{Logging, RDD, SparkContext} import spark.mllib.optimization._ import spark.mllib.util.MLUtils +import scala.math.round + import org.jblas.DoubleMatrix /** @@ -42,14 +44,14 @@ class LogisticRegressionModel( val localIntercept = intercept testData.map { x => val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept - (1.0/ (1.0 + math.exp(margin * -1))).toInt + round(1.0/ (1.0 + math.exp(margin * -1))).toInt } } override def predict(testData: Array[Double]): Int = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept - (1.0/ (1.0 + math.exp(margin * -1))).toInt + round(1.0/ (1.0 + math.exp(margin * -1))).toInt } } From 3a8d07df8ca5bccdbed178991dd12fde74802542 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:20:26 -0700 Subject: [PATCH 199/221] Deleting extra LogisticRegressionGenerator and RidgeRegressionGenerator --- .../LogisticRegressionGenerator.scala | 41 -------------- .../regression/RidgeRegressionGenerator.scala | 55 ------------------- 2 files changed, 96 deletions(-) delete mode 100644 mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala deleted file mode 100644 index cde1148adf..0000000000 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala +++ /dev/null @@ -1,41 +0,0 @@ -package spark.mllib.classification - -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils - -object LogisticRegressionGenerator { - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: LogisticRegressionGenerator " + - " ") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 - val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 - val parts: Int = if (args.length > 4) args(4).toInt else 2 - val eps = 3 - - val sc = new SparkContext(sparkMaster, "LogisticRegressionGenerator") - - val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => - val rnd = new Random(42 + idx) - - val y = if (idx % 2 == 0) 0 else 1 - val x = Array.fill[Double](nfeatures) { - rnd.nextGaussian() + (y * eps) - } - (y, x) - } - - MLUtils.saveLabeledData(data, outputPath) - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala deleted file mode 100644 index b83f505d8e..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ /dev/null @@ -1,55 +0,0 @@ -package spark.mllib.regression - -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils - - -object RidgeRegressionGenerator { - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: RidgeRegressionGenerator " + - " ") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 - val nfeatures: Int = if (args.length > 3) args(3).toInt else 100 - val parts: Int = if (args.length > 4) args(4).toInt else 2 - val eps = 10 - - org.jblas.util.Random.seed(42) - val sc = new SparkContext(sparkMaster, "RidgeRegressionGenerator") - - // Random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - w.put(0, 0, 10) - w.put(1, 0, 10) - - val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until parts, parts).flatMap { p => - org.jblas.util.Random.seed(42 + p) - val examplesInPartition = nexamples / parts - - val X = DoubleMatrix.rand(examplesInPartition, nfeatures) - val y = X.mmul(w) - - val rnd = new Random(42 + p) - - val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) - val yObs = new DoubleMatrix(normalValues).addi(y) - - Iterator.tabulate(examplesInPartition) { i => - (yObs.get(i, 0), X.getRow(i).toArray) - } - } - - MLUtils.saveLabeledData(data, outputPath) - sc.stop() - } -} From 07f17439a52b65d4f5ef8c8d80bc25dadc0182a8 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:22:31 -0700 Subject: [PATCH 200/221] Fix validatePrediction functions for Classification models Classifiers return categorical (Int) values that should be compared directly --- .../spark/mllib/classification/LogisticRegressionSuite.scala | 3 +-- mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 3aa9fe6d12..d3fe58a382 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -62,8 +62,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => - // A prediction is off if the prediction is more than 0.5 away from expected value. - math.abs(prediction.toDouble - expected.toDouble) > 0.5 + (prediction != expected) }.size // At least 80% of the predictions should be on. assert(numOffPredictions < input.length / 5) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 3f00398a0a..d546e0729e 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -52,8 +52,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => - // A prediction is off if the prediction is more than 0.5 away from expected value. - math.abs(prediction - expected) > 0.5 + (prediction != expected) }.size // At least 80% of the predictions should be on. assert(numOffPredictions < input.length / 5) From 2b2630ba3c621f4121da8e76fe6fcfa69d3eb74c Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:22:49 -0700 Subject: [PATCH 201/221] Style fix Lines shortened to < 100 characters --- mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala | 3 ++- mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala index 1ff32d2e7f..ef4f42a494 100644 --- a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala @@ -26,7 +26,8 @@ object LassoGenerator { val sc = new SparkContext(sparkMaster, "LassoGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) + val trueWeights = new DoubleMatrix(1, nfeatures+1, + Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala index ced52093f5..00a54d9a70 100644 --- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala @@ -29,7 +29,8 @@ object SVMGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) + val trueWeights = new DoubleMatrix(1, nfeatures+1, + Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) From 43a2cc15c0539c14732dc10220fc8bb2f152a111 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 13:37:24 -0700 Subject: [PATCH 202/221] Use Bootstrap progress bars in web UI --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index f31af3cda6..16390708ca 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -76,16 +76,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { } def makeProgressBar(completed: Int, total: Int): Seq[Node] = { - val width=130 - val height=15 - val completeWidth = (completed.toDouble / total) * width + val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) - - - - +
    +
    +
    } From 478a2886d95fe2fcaaad07790c9c422736b539ce Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 14:51:07 -0700 Subject: [PATCH 203/221] Added started tasks to progress bar --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 16390708ca..f7f5c2fb6e 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -75,11 +75,13 @@ private[spark] class IndexPage(parent: JobProgressUI) { } } - def makeProgressBar(completed: Int, total: Int): Seq[Node] = { + def makeProgressBar(started: Int, completed: Int, total: Int): Seq[Node] = { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + val startWidth = "width: %s%%".format((started.toDouble/total)*100)
    -
    +
    +
    } @@ -96,6 +98,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { case (false, true) => "Write" case _ => "" } + val startedTasks = listener.stageToTasksActive.getOrElse(s.id, Seq[Long]()).size val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions @@ -105,7 +108,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeProgressBar(completedTasks, totalTasks)} + {makeProgressBar(startedTasks, completedTasks, totalTasks)} {completedTasks} / {totalTasks} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) From 2d6da9195ab8feeaf3fc478ec09d22568d6aa64b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 15:50:52 -0700 Subject: [PATCH 204/221] Alphabetized imports --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index c12a4fe2a5..646ae5ecbc 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -25,9 +25,9 @@ import scala.Some import scala.xml.{NodeSeq, Node} import spark.scheduler.Stage -import spark.ui.UIUtils._ -import spark.ui.Page._ import spark.storage.StorageLevel +import spark.ui.Page._ +import spark.ui.UIUtils._ import spark.Utils /** Page showing list of all ongoing and recently finished stages */ From 07da72b45190f7db9daa2c6bd33577d28e19e659 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 29 Jul 2013 16:25:17 -0700 Subject: [PATCH 205/221] Remove duplicate loss history and clarify why. Also some minor style fixes. --- .../mllib/classification/LogisticRegression.scala | 5 +---- .../mllib/optimization/GradientDescent.scala | 15 +++++++-------- .../scala/spark/mllib/optimization/Updater.scala | 2 +- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index bc1c327729..bf3b05dedb 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -151,7 +151,6 @@ object LogisticRegressionLocalRandomSGD { input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, - miniBatchFraction: Double, initialWeights: Array[Double]) : LogisticRegressionModel = @@ -174,7 +173,6 @@ object LogisticRegressionLocalRandomSGD { input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, - miniBatchFraction: Double) : LogisticRegressionModel = { @@ -195,8 +193,7 @@ object LogisticRegressionLocalRandomSGD { def train( input: RDD[(Int, Array[Double])], numIterations: Int, - stepSize: Double - ) + stepSize: Double) : LogisticRegressionModel = { train(input, numIterations, stepSize, 1.0) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index d4b83a1456..19cda26446 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -61,7 +61,7 @@ object GradientDescent { // Initialize weights as a column vector var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) - var reg_val = 0.0 + var regVal = 0.0 for (i <- 1 to numIters) { val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { @@ -71,15 +71,14 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) + /** + * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration + * and regVal is the regularization value computed in the previous iteration as well. + */ + stochasticLossHistory.append(lossSum / miniBatchSize + regVal) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) weights = update._1 - reg_val = update._2 - stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) - /* - * NOTE(Xinghao): The loss here is sum of lossSum computed using the weights before applying updater, - * and reg_val using weights after applying updater - */ + regVal = update._2 } (weights.toArray, stochasticLossHistory.toArray) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index 188fe7d972..bbf21e5c28 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -76,7 +76,7 @@ class SquaredL2Updater extends Updater { val thisIterStepSize = stepSize / math.sqrt(iter) val normGradient = gradient.mul(thisIterStepSize) val newWeights = weightsOld.sub(normGradient).div(2.0 * thisIterStepSize * regParam + 1.0) - (newWeights, pow(newWeights.norm2,2.0) * regParam) + (newWeights, pow(newWeights.norm2, 2.0) * regParam) } } From 87b821dc3953bac7584592b9b030c2f57203dd0d Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 16:50:51 -0700 Subject: [PATCH 206/221] Fixed continuity of executorToTasksActive, changed color of progress bars --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index b70153fd30..d6f83157d3 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -93,7 +93,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, Seq[Long]()).size.toString + val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, HashSet[TaskInfo]()).size.toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index e0a4168290..e2cf84346a 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -21,9 +21,11 @@ import java.util.Date import javax.servlet.http.HttpServletRequest +import scala.collection.mutable.HashSet import scala.Some import scala.xml.{NodeSeq, Node} +import spark.scheduler.cluster.TaskInfo import spark.scheduler.Stage import spark.storage.StorageLevel import spark.ui.Page._ @@ -110,8 +112,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { val startWidth = "width: %s%%".format((started.toDouble/total)*100)
    -
    -
    +
    +
    } @@ -131,6 +133,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { case b => Utils.memoryBytesToString(b) } + val startedTasks = listener.stageToTasksActive.getOrElse(s.id, HashSet[TaskInfo]()).size val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions From 17e62113d446c87d0313daa2ac230867591627be Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:14:44 -0700 Subject: [PATCH 207/221] Moved DeployMessage's into its own DeployMessages object. Also renamed MasterState to MasterStateResponse and WorkerState to WorkerStateResponse for clarity. --- .../scala/spark/deploy/DeployMessage.scala | 172 +++++++++--------- .../scala/spark/deploy/JsonProtocol.scala | 11 +- .../scala/spark/deploy/client/Client.scala | 18 +- .../scala/spark/deploy/master/Master.scala | 39 ++-- .../deploy/master/ui/ApplicationPage.scala | 11 +- .../spark/deploy/master/ui/IndexPage.scala | 16 +- .../spark/deploy/worker/ExecutorRunner.scala | 10 +- .../scala/spark/deploy/worker/Worker.scala | 30 +-- .../spark/deploy/worker/ui/IndexPage.scala | 23 ++- 9 files changed, 174 insertions(+), 156 deletions(-) diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index e1f8aff6f5..7c37a16615 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -17,109 +17,107 @@ package spark.deploy +import scala.collection.immutable.List + +import spark.Utils import spark.deploy.ExecutorState.ExecutorState import spark.deploy.master.{WorkerInfo, ApplicationInfo} import spark.deploy.worker.ExecutorRunner -import scala.collection.immutable.List -import spark.Utils -private[spark] sealed trait DeployMessage extends Serializable +private[deploy] sealed trait DeployMessage extends Serializable -// Worker to Master +private[deploy] object DeployMessages { -private[spark] -case class RegisterWorker( - id: String, - host: String, - port: Int, - cores: Int, - memory: Int, - webUiPort: Int, - publicAddress: String) - extends DeployMessage { - Utils.checkHost(host, "Required hostname") - assert (port > 0) -} + // Worker to Master -private[spark] -case class ExecutorStateChanged( - appId: String, - execId: Int, - state: ExecutorState, - message: Option[String], + case class RegisterWorker( + id: String, + host: String, + port: Int, + cores: Int, + memory: Int, + webUiPort: Int, + publicAddress: String) + extends DeployMessage { + Utils.checkHost(host, "Required hostname") + assert (port > 0) + } + + case class ExecutorStateChanged( + appId: String, + execId: Int, + state: ExecutorState, + message: Option[String], + exitStatus: Option[Int]) + extends DeployMessage + + case class Heartbeat(workerId: String) extends DeployMessage + + // Master to Worker + + case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage + + case class RegisterWorkerFailed(message: String) extends DeployMessage + + case class KillExecutor(appId: String, execId: Int) extends DeployMessage + + case class LaunchExecutor( + appId: String, + execId: Int, + appDesc: ApplicationDescription, + cores: Int, + memory: Int, + sparkHome: String) + extends DeployMessage + + // Client to Master + + case class RegisterApplication(appDescription: ApplicationDescription) + extends DeployMessage + + // Master to Client + + case class RegisteredApplication(appId: String) extends DeployMessage + + case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { + Utils.checkHostPort(hostPort, "Required hostport") + } + + case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String], exitStatus: Option[Int]) - extends DeployMessage -private[spark] case class Heartbeat(workerId: String) extends DeployMessage + case class ApplicationRemoved(message: String) -// Master to Worker + // Internal message in Client -private[spark] case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage -private[spark] case class RegisterWorkerFailed(message: String) extends DeployMessage -private[spark] case class KillExecutor(appId: String, execId: Int) extends DeployMessage + case object StopClient -private[spark] case class LaunchExecutor( - appId: String, - execId: Int, - appDesc: ApplicationDescription, - cores: Int, - memory: Int, - sparkHome: String) - extends DeployMessage + // MasterWebUI To Master -// Client to Master + case object RequestMasterState -private[spark] case class RegisterApplication(appDescription: ApplicationDescription) - extends DeployMessage + // Master to MasterWebUI -// Master to Client + case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo], + activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) { -private[spark] -case class RegisteredApplication(appId: String) extends DeployMessage + Utils.checkHost(host, "Required hostname") + assert (port > 0) -private[spark] -case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { - Utils.checkHostPort(hostPort, "Required hostport") -} - -private[spark] -case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String], - exitStatus: Option[Int]) - -private[spark] -case class ApplicationRemoved(message: String) - -// Internal message in Client - -private[spark] case object StopClient - -// MasterWebUI To Master - -private[spark] case object RequestMasterState - -// Master to MasterWebUI - -private[spark] -case class MasterState(host: String, port: Int, workers: Array[WorkerInfo], - activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) { - - Utils.checkHost(host, "Required hostname") - assert (port > 0) - - def uri = "spark://" + host + ":" + port -} - -// WorkerWebUI to Worker -private[spark] case object RequestWorkerState - -// Worker to WorkerWebUI - -private[spark] -case class WorkerState(host: String, port: Int, workerId: String, executors: List[ExecutorRunner], - finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int, - coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) { - - Utils.checkHost(host, "Required hostname") - assert (port > 0) + def uri = "spark://" + host + ":" + port + } + + // WorkerWebUI to Worker + case object RequestWorkerState + + // Worker to WorkerWebUI + + case class WorkerStateResponse(host: String, port: Int, workerId: String, + executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String, + cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) { + + Utils.checkHost(host, "Required hostname") + assert (port > 0) + } } diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 64f89623e1..bd1db7c294 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -17,9 +17,12 @@ package spark.deploy -import master.{ApplicationInfo, WorkerInfo} import net.liftweb.json.JsonDSL._ -import worker.ExecutorRunner + +import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} +import spark.deploy.master.{ApplicationInfo, WorkerInfo} +import spark.deploy.worker.ExecutorRunner + private[spark] object JsonProtocol { def writeWorkerInfo(obj: WorkerInfo) = { @@ -57,7 +60,7 @@ private[spark] object JsonProtocol { ("appdesc" -> writeApplicationDescription(obj.appDesc)) } - def writeMasterState(obj: MasterState) = { + def writeMasterState(obj: MasterStateResponse) = { ("url" -> ("spark://" + obj.uri)) ~ ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ ("cores" -> obj.workers.map(_.cores).sum) ~ @@ -68,7 +71,7 @@ private[spark] object JsonProtocol { ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) } - def writeWorkerState(obj: WorkerState) = { + def writeWorkerState(obj: WorkerStateResponse) = { ("id" -> obj.workerId) ~ ("masterurl" -> obj.masterUrl) ~ ("masterwebuiurl" -> obj.masterWebUiUrl) ~ diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 29e494f495..9d5ba8a796 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -17,21 +17,23 @@ package spark.deploy.client -import spark.deploy._ +import java.util.concurrent.TimeoutException + import akka.actor._ +import akka.actor.Terminated import akka.pattern.ask import akka.util.Duration -import akka.util.duration._ -import java.util.concurrent.TimeoutException -import spark.{SparkException, Logging} +import akka.remote.RemoteClientDisconnected import akka.remote.RemoteClientLifeCycleEvent import akka.remote.RemoteClientShutdown -import spark.deploy.RegisterApplication -import spark.deploy.master.Master -import akka.remote.RemoteClientDisconnected -import akka.actor.Terminated import akka.dispatch.Await +import spark.Logging +import spark.deploy.{ApplicationDescription, ExecutorState} +import spark.deploy.DeployMessages._ +import spark.deploy.master.Master + + /** * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description, * and a listener for cluster events, and calls back the listener when various events occur. diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 9692af5295..202d5bcdb7 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -17,21 +17,22 @@ package spark.deploy.master -import akka.actor._ -import akka.actor.Terminated -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} -import akka.util.duration._ - import java.text.SimpleDateFormat import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import spark.deploy._ +import akka.actor._ +import akka.actor.Terminated +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} +import akka.util.duration._ + import spark.{Logging, SparkException, Utils} +import spark.deploy.{ApplicationDescription, ExecutorState} +import spark.deploy.DeployMessages._ +import spark.deploy.master.ui.MasterWebUI import spark.metrics.MetricsSystem import spark.util.AkkaUtils -import ui.MasterWebUI private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { @@ -168,7 +169,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } case RequestMasterState => { - sender ! MasterState(host, port, workers.toArray, apps.toArray, completedApps.toArray) + sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray) } } @@ -233,20 +234,27 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) - worker.actor ! LaunchExecutor(exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome) - exec.application.driver ! ExecutorAdded(exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) + worker.actor ! LaunchExecutor( + exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome) + exec.application.driver ! ExecutorAdded( + exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) } def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int, publicAddress: String): WorkerInfo = { - // There may be one or more refs to dead workers on this same node (w/ different ID's), remove them. - workers.filter(w => (w.host == host && w.port == port) && (w.state == WorkerState.DEAD)).foreach(workers -= _) + // There may be one or more refs to dead workers on this same node (w/ different ID's), + // remove them. + workers.filter { w => + (w.host == host && w.port == port) && (w.state == WorkerState.DEAD) + }.foreach { w => + workers -= w + } val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress) workers += worker idToWorker(worker.id) = worker actorToWorker(sender) = worker addressToWorker(sender.path.address) = worker - return worker + worker } def removeWorker(worker: WorkerInfo) { @@ -257,7 +265,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act addressToWorker -= worker.actor.path.address for (exec <- worker.executors.values) { logInfo("Telling app of lost executor: " + exec.id) - exec.application.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None) + exec.application.driver ! ExecutorUpdated( + exec.id, ExecutorState.LOST, Some("worker lost"), None) exec.application.removeExecutor(exec) } } @@ -277,7 +286,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= desc.memoryPerSlave)) { logWarning("Could not find any workers with enough memory for " + firstApp.get.id) } - return app + app } def finishApplication(app: ApplicationInfo) { diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 32264af393..b4c62bc224 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -17,6 +17,8 @@ package spark.deploy.master.ui +import scala.xml.Node + import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ @@ -25,9 +27,8 @@ import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import scala.xml.Node - -import spark.deploy.{RequestMasterState, JsonProtocol, MasterState} +import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import spark.deploy.JsonProtocol import spark.deploy.master.ExecutorInfo import spark.ui.UIUtils @@ -38,7 +39,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { /** Executor details for a particular application */ def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) @@ -49,7 +50,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { /** Executor details for a particular application */ def render(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index b05197c1b9..557df89b41 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -17,18 +17,20 @@ package spark.deploy.master.ui -import akka.dispatch.Await -import akka.pattern.ask -import akka.util.duration._ - import javax.servlet.http.HttpServletRequest import scala.xml.Node -import spark.deploy.{RequestMasterState, DeployWebUI, MasterState} +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.duration._ + import spark.Utils -import spark.ui.UIUtils +import spark.deploy.DeployWebUI +import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import spark.deploy.master.{ApplicationInfo, WorkerInfo} +import spark.ui.UIUtils + private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.master @@ -36,7 +38,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 47d3390928..345dfe879c 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -19,14 +19,12 @@ package spark.deploy.worker import java.io._ import java.lang.System.getenv -import spark.deploy.{ExecutorState, ExecutorStateChanged, ApplicationDescription} + import akka.actor.ActorRef + import spark.{Utils, Logging} -import java.net.{URI, URL} -import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.hadoop.conf.Configuration -import scala.Some -import spark.deploy.ExecutorStateChanged +import spark.deploy.{ExecutorState, ApplicationDescription} +import spark.deploy.DeployMessages.ExecutorStateChanged /** * Manages the execution of one executor process. diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 8fa0d12b82..0e46fa281e 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -17,22 +17,24 @@ package spark.deploy.worker -import scala.collection.mutable.{ArrayBuffer, HashMap} -import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import akka.util.duration._ -import spark.{Logging, Utils} -import spark.util.AkkaUtils -import spark.deploy._ -import spark.metrics.MetricsSystem -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.text.SimpleDateFormat import java.util.Date -import spark.deploy.RegisterWorker -import spark.deploy.LaunchExecutor -import spark.deploy.RegisterWorkerFailed -import spark.deploy.master.Master import java.io.File -import ui.WorkerWebUI + +import scala.collection.mutable.HashMap + +import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} +import akka.util.duration._ + +import spark.{Logging, Utils} +import spark.deploy.ExecutorState +import spark.deploy.DeployMessages._ +import spark.deploy.master.Master +import spark.deploy.worker.ui.WorkerWebUI +import spark.metrics.MetricsSystem +import spark.util.AkkaUtils + private[spark] class Worker( host: String, @@ -164,7 +166,7 @@ private[spark] class Worker( masterDisconnected() case RequestWorkerState => { - sender ! WorkerState(host, port, workerId, executors.values.toList, + sender ! WorkerStateResponse(host, port, workerId, executors.values.toList, finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed, masterWebUiUrl) } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 7548a26c2e..1619c6a4c2 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -17,34 +17,36 @@ package spark.deploy.worker.ui +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ -import javax.servlet.http.HttpServletRequest - import net.liftweb.json.JsonAST.JValue -import scala.xml.Node - -import spark.deploy.{RequestWorkerState, JsonProtocol, WorkerState} -import spark.deploy.worker.ExecutorRunner import spark.Utils +import spark.deploy.JsonProtocol +import spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} +import spark.deploy.worker.ExecutorRunner import spark.ui.UIUtils + private[spark] class IndexPage(parent: WorkerWebUI) { val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout def renderJson(request: HttpServletRequest): JValue = { - val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, 30 seconds) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, 30 seconds) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") @@ -69,7 +71,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
    -
    +
    @@ -88,7 +90,8 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
    ; - UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format( + workerState.host, workerState.port)) } def executorRow(executor: ExecutorRunner): Seq[Node] = { From 207548b67bc6ce208b60215d40a03904d8ac0cbc Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 17:19:33 -0700 Subject: [PATCH 208/221] Open up Job UI ports (33000-33010) on EC2 clusters --- ec2/spark_ec2.py | 39 ++++++++++++++++++++------------------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 2ec3c007fb..740ec08542 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -9,9 +9,9 @@ # 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. @@ -53,7 +53,7 @@ def parse_args(): help="Seconds to wait for nodes to start (default: 120)") parser.add_option("-k", "--key-pair", help="Key pair to use on instances") - parser.add_option("-i", "--identity-file", + parser.add_option("-i", "--identity-file", help="SSH private key file to use for logging into instances") parser.add_option("-t", "--instance-type", default="m1.large", help="Type of instance to launch (default: m1.large). " + @@ -69,7 +69,7 @@ def parse_args(): parser.add_option("-a", "--ami", default="latest", help="Amazon Machine Image ID to use, or 'latest' to use latest " + "available AMI (default: latest)") - parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port", + parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port", help="Use SSH dynamic port forwarding to create a SOCKS proxy at " + "the given local address (for use with login)") parser.add_option("--resume", action="store_true", default=False, @@ -99,7 +99,7 @@ def parse_args(): help="The SSH user you want to connect as (default: root)") parser.add_option("--delete-groups", action="store_true", default=False, help="When destroying a cluster, delete the security groups that were created") - + (opts, args) = parser.parse_args() if len(args) != 2: parser.print_help() @@ -112,7 +112,7 @@ def parse_args(): if opts.cluster_type not in ["mesos", "standalone"] and action == "launch": print >> stderr, ("ERROR: Invalid cluster type: " + opts.cluster_type) sys.exit(1) - + # Boto config check # http://boto.cloudhackers.com/en/latest/boto_config_tut.html home_dir = os.getenv('HOME') @@ -178,6 +178,7 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') + master_group.authorize('tcp', 33000, 33010, '0.0.0.0/0') if opts.cluster_type == "mesos": master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0') if opts.ganglia: @@ -257,7 +258,7 @@ def launch_cluster(conn, opts, cluster_name): block_device_map = block_map) my_req_ids += [req.id for req in slave_reqs] i += 1 - + print "Waiting for spot instances to be granted..." try: while True: @@ -413,7 +414,7 @@ def setup_standalone_cluster(master, slave_nodes, opts): slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes]) ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips)) ssh(master, opts, "/root/spark/bin/start-all.sh") - + def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") @@ -528,7 +529,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes, dest.write(text) dest.close() # rsync the whole directory over to the master machine - command = (("rsync -rv -e 'ssh -o StrictHostKeyChecking=no -i %s' " + + command = (("rsync -rv -e 'ssh -o StrictHostKeyChecking=no -i %s' " + "'%s/' '%s@%s:/'") % (opts.identity_file, tmp_dir, opts.user, active_master)) subprocess.check_call(command, shell=True) # Remove the temp directory we created above @@ -557,9 +558,9 @@ def ssh(host, opts, command): print "Error connecting to host {0}, sleeping 30".format(e) time.sleep(30) tries = tries + 1 - - - + + + # Gets a list of zones to launch instances in @@ -618,12 +619,12 @@ def main(): print "Terminating zoo..." for inst in zoo_nodes: inst.terminate() - + # Delete security groups as well if opts.delete_groups: print "Deleting security groups (this will take some time)..." group_names = [cluster_name + "-master", cluster_name + "-slaves", cluster_name + "-zoo"] - + attempt = 1; while attempt <= 3: print "Attempt %d" % attempt @@ -639,7 +640,7 @@ def main(): from_port=rule.from_port, to_port=rule.to_port, src_group=grant) - + # Sleep for AWS eventual-consistency to catch up, and for instances # to terminate time.sleep(30) # Yes, it does have to be this long :-( @@ -650,13 +651,13 @@ def main(): except boto.exception.EC2ResponseError: success = False; print "Failed to delete security group " + group.name - + # Unfortunately, group.revoke() returns True even if a rule was not # deleted, so this needs to be rerun if something fails if success: break; - + attempt += 1 - + if not success: print "Failed to delete all security groups after 3 tries." print "Try re-running in a few minutes." @@ -679,7 +680,7 @@ def main(): elif action == "stop": response = raw_input("Are you sure you want to stop the cluster " + cluster_name + "?\nDATA ON EPHEMERAL DISKS WILL BE LOST, " + - "BUT THE CLUSTER WILL KEEP USING SPACE ON\n" + + "BUT THE CLUSTER WILL KEEP USING SPACE ON\n" + "AMAZON EBS IF IT IS EBS-BACKED!!\n" + "Stop cluster " + cluster_name + " (y/N): ") if response == "y": From 105f4d22e93af23ff3bf117bcfd976d412f494e6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:30:38 -0700 Subject: [PATCH 209/221] Removed Cache and SoftReferenceCache since they are no longer used. --- core/src/main/scala/spark/Cache.scala | 80 ------------------- .../main/scala/spark/SoftReferenceCache.scala | 35 -------- 2 files changed, 115 deletions(-) delete mode 100644 core/src/main/scala/spark/Cache.scala delete mode 100644 core/src/main/scala/spark/SoftReferenceCache.scala diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala deleted file mode 100644 index b0c83ce59d..0000000000 --- a/core/src/main/scala/spark/Cache.scala +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package spark - -import java.util.concurrent.atomic.AtomicInteger - -private[spark] sealed trait CachePutResponse -private[spark] case class CachePutSuccess(size: Long) extends CachePutResponse -private[spark] case class CachePutFailure() extends CachePutResponse - -/** - * An interface for caches in Spark, to allow for multiple implementations. Caches are used to store - * both partitions of cached RDDs and broadcast variables on Spark executors. Caches are also aware - * of which entries are part of the same dataset (for example, partitions in the same RDD). The key - * for each value in a cache is a (datasetID, partition) pair. - * - * A single Cache instance gets created on each machine and is shared by all caches (i.e. both the - * RDD split cache and the broadcast variable cache), to enable global replacement policies. - * However, because these several independent modules all perform caching, it is important to give - * them separate key namespaces, so that an RDD and a broadcast variable (for example) do not use - * the same key. For this purpose, Cache has the notion of KeySpaces. Each client module must first - * ask for a KeySpace, and then call get() and put() on that space using its own keys. - * - * This abstract class handles the creation of key spaces, so that subclasses need only deal with - * keys that are unique across modules. - */ -private[spark] abstract class Cache { - private val nextKeySpaceId = new AtomicInteger(0) - private def newKeySpaceId() = nextKeySpaceId.getAndIncrement() - - def newKeySpace() = new KeySpace(this, newKeySpaceId()) - - /** - * Get the value for a given (datasetId, partition), or null if it is not - * found. - */ - def get(datasetId: Any, partition: Int): Any - - /** - * Attempt to put a value in the cache; returns CachePutFailure if this was - * not successful (e.g. because the cache replacement policy forbids it), and - * CachePutSuccess if successful. If size estimation is available, the cache - * implementation should set the size field in CachePutSuccess. - */ - def put(datasetId: Any, partition: Int, value: Any): CachePutResponse - - /** - * Report the capacity of the cache partition. By default this just reports - * zero. Specific implementations can choose to provide the capacity number. - */ - def getCapacity: Long = 0L -} - -/** - * A key namespace in a Cache. - */ -private[spark] class KeySpace(cache: Cache, val keySpaceId: Int) { - def get(datasetId: Any, partition: Int): Any = - cache.get((keySpaceId, datasetId), partition) - - def put(datasetId: Any, partition: Int, value: Any): CachePutResponse = - cache.put((keySpaceId, datasetId), partition, value) - - def getCapacity: Long = cache.getCapacity -} diff --git a/core/src/main/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala deleted file mode 100644 index f41a379582..0000000000 --- a/core/src/main/scala/spark/SoftReferenceCache.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package spark - -import com.google.common.collect.MapMaker - -/** - * An implementation of Cache that uses soft references. - */ -private[spark] class SoftReferenceCache extends Cache { - val map = new MapMaker().softValues().makeMap[Any, Any]() - - override def get(datasetId: Any, partition: Int): Any = - map.get((datasetId, partition)) - - override def put(datasetId: Any, partition: Int, value: Any): CachePutResponse = { - map.put((datasetId, partition), value) - return CachePutSuccess(0) - } -} From 23b5da14ed6413b0dcb4c0bfdb80c98c433f3c9d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:42:05 -0700 Subject: [PATCH 210/221] Moved block manager messages into BlockManagerMessages object. --- .../spark/storage/BlockManagerMaster.scala | 1 + .../storage/BlockManagerMasterActor.scala | 2 + .../spark/storage/BlockManagerMessages.scala | 159 ++++++++---------- .../storage/BlockManagerSlaveActor.scala | 2 +- .../spark/storage/BlockManagerSource.scala | 2 +- .../scala/spark/storage/BlockMessage.scala | 1 - .../spark/storage/BlockMessageArray.scala | 5 +- .../spark/storage/BlockObjectWriter.scala | 2 - 8 files changed, 80 insertions(+), 94 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 3186f7c85b..76128e8cff 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -23,6 +23,7 @@ import akka.pattern.ask import akka.util.Duration import spark.{Logging, SparkException} +import spark.storage.BlockManagerMessages._ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 244000d952..011bb6b83d 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -29,6 +29,8 @@ import akka.util.Duration import akka.util.duration._ import spark.{Logging, Utils, SparkException} +import spark.storage.BlockManagerMessages._ + /** * BlockManagerMasterActor is an actor on the master node to track statuses of diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 01de4ccb8f..9375a9ca54 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -22,102 +22,89 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import akka.actor.ActorRef -////////////////////////////////////////////////////////////////////////////////// -// Messages from the master to slaves. -////////////////////////////////////////////////////////////////////////////////// -private[spark] -sealed trait ToBlockManagerSlave +private[storage] object BlockManagerMessages { + ////////////////////////////////////////////////////////////////////////////////// + // Messages from the master to slaves. + ////////////////////////////////////////////////////////////////////////////////// + sealed trait ToBlockManagerSlave -// Remove a block from the slaves that have it. This can only be used to remove -// blocks that the master knows about. -private[spark] -case class RemoveBlock(blockId: String) extends ToBlockManagerSlave + // Remove a block from the slaves that have it. This can only be used to remove + // blocks that the master knows about. + case class RemoveBlock(blockId: String) extends ToBlockManagerSlave -// Remove all blocks belonging to a specific RDD. -private[spark] case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave + // Remove all blocks belonging to a specific RDD. + case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave -////////////////////////////////////////////////////////////////////////////////// -// Messages from slaves to the master. -////////////////////////////////////////////////////////////////////////////////// -private[spark] -sealed trait ToBlockManagerMaster + ////////////////////////////////////////////////////////////////////////////////// + // Messages from slaves to the master. + ////////////////////////////////////////////////////////////////////////////////// + sealed trait ToBlockManagerMaster -private[spark] -case class RegisterBlockManager( - blockManagerId: BlockManagerId, - maxMemSize: Long, - sender: ActorRef) - extends ToBlockManagerMaster + case class RegisterBlockManager( + blockManagerId: BlockManagerId, + maxMemSize: Long, + sender: ActorRef) + extends ToBlockManagerMaster -private[spark] -case class HeartBeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster + case class HeartBeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster -private[spark] -class UpdateBlockInfo( - var blockManagerId: BlockManagerId, - var blockId: String, - var storageLevel: StorageLevel, - var memSize: Long, - var diskSize: Long) - extends ToBlockManagerMaster - with Externalizable { + class UpdateBlockInfo( + var blockManagerId: BlockManagerId, + var blockId: String, + var storageLevel: StorageLevel, + var memSize: Long, + var diskSize: Long) + extends ToBlockManagerMaster + with Externalizable { - def this() = this(null, null, null, 0, 0) // For deserialization only + def this() = this(null, null, null, 0, 0) // For deserialization only - override def writeExternal(out: ObjectOutput) { - blockManagerId.writeExternal(out) - out.writeUTF(blockId) - storageLevel.writeExternal(out) - out.writeLong(memSize) - out.writeLong(diskSize) + override def writeExternal(out: ObjectOutput) { + blockManagerId.writeExternal(out) + out.writeUTF(blockId) + storageLevel.writeExternal(out) + out.writeLong(memSize) + out.writeLong(diskSize) + } + + override def readExternal(in: ObjectInput) { + blockManagerId = BlockManagerId(in) + blockId = in.readUTF() + storageLevel = StorageLevel(in) + memSize = in.readLong() + diskSize = in.readLong() + } } - override def readExternal(in: ObjectInput) { - blockManagerId = BlockManagerId(in) - blockId = in.readUTF() - storageLevel = StorageLevel(in) - memSize = in.readLong() - diskSize = in.readLong() + object UpdateBlockInfo { + def apply(blockManagerId: BlockManagerId, + blockId: String, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long): UpdateBlockInfo = { + new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) + } + + // For pattern-matching + def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = { + Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) + } } + + case class GetLocations(blockId: String) extends ToBlockManagerMaster + + case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster + + case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster + + case class RemoveExecutor(execId: String) extends ToBlockManagerMaster + + case object StopBlockManagerMaster extends ToBlockManagerMaster + + case object GetMemoryStatus extends ToBlockManagerMaster + + case object ExpireDeadHosts extends ToBlockManagerMaster + + case object GetStorageStatus extends ToBlockManagerMaster } - -private[spark] -object UpdateBlockInfo { - def apply(blockManagerId: BlockManagerId, - blockId: String, - storageLevel: StorageLevel, - memSize: Long, - diskSize: Long): UpdateBlockInfo = { - new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) - } - - // For pattern-matching - def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) - } -} - -private[spark] -case class GetLocations(blockId: String) extends ToBlockManagerMaster - -private[spark] -case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster - -private[spark] -case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster - -private[spark] -case class RemoveExecutor(execId: String) extends ToBlockManagerMaster - -private[spark] -case object StopBlockManagerMaster extends ToBlockManagerMaster - -private[spark] -case object GetMemoryStatus extends ToBlockManagerMaster - -private[spark] -case object ExpireDeadHosts extends ToBlockManagerMaster - -private[spark] -case object GetStorageStatus extends ToBlockManagerMaster diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala index 45cffad810..6e5fb43732 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala @@ -19,7 +19,7 @@ package spark.storage import akka.actor.Actor -import spark.{Logging, SparkException, Utils} +import spark.storage.BlockManagerMessages._ /** diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index 4faa715c94..2aecd1ea71 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -3,7 +3,7 @@ package spark.storage import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -import spark.storage._ + private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala index ab72dbb62b..bcce26b7c1 100644 --- a/core/src/main/scala/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/spark/storage/BlockMessage.scala @@ -22,7 +22,6 @@ import java.nio.ByteBuffer import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer -import spark._ import spark.network._ private[spark] case class GetBlock(id: String) diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala index b0229d6124..ee2fc167d5 100644 --- a/core/src/main/scala/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala @@ -19,7 +19,6 @@ package spark.storage import java.nio.ByteBuffer -import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer import spark._ @@ -113,7 +112,7 @@ private[spark] object BlockMessageArray { def main(args: Array[String]) { val blockMessages = - (0 until 10).map(i => { + (0 until 10).map { i => if (i % 2 == 0) { val buffer = ByteBuffer.allocate(100) buffer.clear @@ -121,7 +120,7 @@ private[spark] object BlockMessageArray { } else { BlockMessage.fromGetBlock(GetBlock(i.toString)) } - }) + } val blockMessageArray = new BlockMessageArray(blockMessages) println("Block message array created") diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala index 01ed6e8c1f..3812009ca1 100644 --- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -17,8 +17,6 @@ package spark.storage -import java.nio.ByteBuffer - /** * An interface for writing JVM objects to some underlying storage. This interface allows From 81720e13fc9e1f475dd1333babfa08f3f806a5d0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:53:01 -0700 Subject: [PATCH 211/221] Moved all StandaloneClusterMessage's into StandaloneClusterMessages object. --- .../executor/StandaloneExecutorBackend.scala | 17 ++--- .../cluster/StandaloneClusterMessage.scala | 65 ++++++++++--------- .../cluster/StandaloneSchedulerBackend.scala | 9 +-- 3 files changed, 45 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index f4003da732..e47fe50021 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -18,19 +18,16 @@ package spark.executor import java.nio.ByteBuffer -import spark.Logging -import spark.TaskState.TaskState -import spark.util.AkkaUtils + import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} -import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} -import spark.scheduler.cluster._ -import spark.scheduler.cluster.RegisteredExecutor -import spark.scheduler.cluster.LaunchTask -import spark.scheduler.cluster.RegisterExecutorFailed -import spark.scheduler.cluster.RegisterExecutor -import spark.Utils + +import spark.{Logging, Utils} +import spark.TaskState.TaskState import spark.deploy.SparkHadoopUtil +import spark.scheduler.cluster.StandaloneClusterMessages._ +import spark.util.AkkaUtils + private[spark] class StandaloneExecutorBackend( driverUrl: String, diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index ac9e5ef94d..05c29eb72f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -17,46 +17,47 @@ package spark.scheduler.cluster -import spark.TaskState.TaskState import java.nio.ByteBuffer -import spark.util.SerializableBuffer + +import spark.TaskState.TaskState import spark.Utils +import spark.util.SerializableBuffer + private[spark] sealed trait StandaloneClusterMessage extends Serializable -// Driver to executors -private[spark] -case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage +private[spark] object StandaloneClusterMessages { -private[spark] -case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) - extends StandaloneClusterMessage + // Driver to executors + case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage -private[spark] -case class RegisterExecutorFailed(message: String) extends StandaloneClusterMessage + case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) + extends StandaloneClusterMessage -// Executors to driver -private[spark] -case class RegisterExecutor(executorId: String, hostPort: String, cores: Int) - extends StandaloneClusterMessage { - Utils.checkHostPort(hostPort, "Expected host port") -} + case class RegisterExecutorFailed(message: String) extends StandaloneClusterMessage -private[spark] -case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer) - extends StandaloneClusterMessage - -private[spark] -object StatusUpdate { - /** Alternate factory method that takes a ByteBuffer directly for the data field */ - def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer): StatusUpdate = { - StatusUpdate(executorId, taskId, state, new SerializableBuffer(data)) + // Executors to driver + case class RegisterExecutor(executorId: String, hostPort: String, cores: Int) + extends StandaloneClusterMessage { + Utils.checkHostPort(hostPort, "Expected host port") } + + case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, + data: SerializableBuffer) extends StandaloneClusterMessage + + object StatusUpdate { + /** Alternate factory method that takes a ByteBuffer directly for the data field */ + def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer) + : StatusUpdate = { + StatusUpdate(executorId, taskId, state, new SerializableBuffer(data)) + } + } + + // Internal messages in driver + case object ReviveOffers extends StandaloneClusterMessage + + case object StopDriver extends StandaloneClusterMessage + + case class RemoveExecutor(executorId: String, reason: String) extends StandaloneClusterMessage + } - -// Internal messages in driver -private[spark] case object ReviveOffers extends StandaloneClusterMessage -private[spark] case object StopDriver extends StandaloneClusterMessage - -private[spark] case class RemoveExecutor(executorId: String, reason: String) - extends StandaloneClusterMessage diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 03a64e0192..075a7cbf7e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -17,17 +17,18 @@ package spark.scheduler.cluster +import java.util.concurrent.atomic.AtomicInteger + import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import akka.actor._ -import akka.util.duration._ +import akka.dispatch.Await import akka.pattern.ask +import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} import akka.util.Duration import spark.{Utils, SparkException, Logging, TaskState} -import akka.dispatch.Await -import java.util.concurrent.atomic.AtomicInteger -import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import spark.scheduler.cluster.StandaloneClusterMessages._ /** * A standalone scheduler backend, which waits for standalone executors to connect to it through From 3ca9faa341dcddb54f8b2e26b582c08901ea875f Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 29 Jul 2013 18:37:28 -0700 Subject: [PATCH 212/221] Clarify how regVal is computed in Updater docs --- .../spark/mllib/optimization/Updater.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index bbf21e5c28..e916a92c33 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -23,6 +23,7 @@ import org.jblas.DoubleMatrix abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize and iteration number. + * Also returns the regularization value computed using the *updated* weights. * * @param weightsOlds - Column matrix of size nx1 where n is the number of features. * @param gradient - Column matrix of size nx1 where n is the number of features. @@ -31,7 +32,7 @@ abstract class Updater extends Serializable { * @param regParam - Regularization parameter * * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, - * and the second element is the regularization value. + * and the second element is the regularization value computed using updated weights. */ def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) @@ -46,13 +47,13 @@ class SimpleUpdater extends Updater { } /** -* L1 regularization -- corresponding proximal operator is the soft-thresholding function -* That is, each weight component is shrunk towards 0 by shrinkageVal -* If w > shrinkageVal, set weight component to w-shrinkageVal. -* If w < -shrinkageVal, set weight component to w+shrinkageVal. -* If -shrinkageVal < w < shrinkageVal, set weight component to 0. -* Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) -**/ + * L1 regularization -- corresponding proximal operator is the soft-thresholding function + * That is, each weight component is shrunk towards 0 by shrinkageVal + * If w > shrinkageVal, set weight component to w-shrinkageVal. + * If w < -shrinkageVal, set weight component to w+shrinkageVal. + * If -shrinkageVal < w < shrinkageVal, set weight component to 0. + * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) + */ class L1Updater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { From b95732632b5b06d494ebd9e539af136ab3b8490e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 28 Jul 2013 19:24:23 -0700 Subject: [PATCH 213/221] Do not inherit master's PYTHONPATH on workers. This fixes SPARK-832, an issue where PySpark would not work when the master and workers used different SPARK_HOME paths. This change may potentially break code that relied on the master's PYTHONPATH being used on workers. To have custom PYTHONPATH additions used on the workers, users should set a custom PYTHONPATH in spark-env.sh rather than setting it in the shell. --- .../main/scala/spark/api/python/PythonWorkerFactory.scala | 2 ++ python/pyspark/rdd.py | 5 ++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 078ad45ce8..84673470db 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -67,6 +67,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) val workerEnv = pb.environment() workerEnv.putAll(envVars) + val pythonPath = sparkHome + "/python/:" + workerEnv.get("PYTHONPATH") + workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() // Redirect the stderr to ours diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 8734cacb0b..51c2cb9806 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -756,9 +756,8 @@ class PipelinedRDD(RDD): self.ctx._gateway._gateway_client) self.ctx._pickled_broadcast_vars.clear() class_manifest = self._prev_jrdd.classManifest() - env = copy.copy(self.ctx.environment) - env['PYTHONPATH'] = os.environ.get("PYTHONPATH", "") - env = MapConverter().convert(env, self.ctx._gateway._gateway_client) + env = MapConverter().convert(self.ctx.environment, + self.ctx._gateway._gateway_client) python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(), pipe_command, env, self.preservesPartitioning, self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator, class_manifest) From 49be084ed3cca4778ff0602c0bee962a3515338e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 28 Jul 2013 21:21:19 -0700 Subject: [PATCH 214/221] Use File.pathSeparator instead of hardcoding ':'. --- .../src/main/scala/spark/api/python/PythonWorkerFactory.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 84673470db..14f8320678 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -17,7 +17,7 @@ package spark.api.python -import java.io.{DataInputStream, IOException} +import java.io.{File, DataInputStream, IOException} import java.net.{Socket, SocketException, InetAddress} import scala.collection.JavaConversions._ @@ -67,7 +67,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/:" + workerEnv.get("PYTHONPATH") + val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() From e4387ddf5d1a46dfedece73feff4de6a30f9a220 Mon Sep 17 00:00:00 2001 From: Ameet Talwalkar Date: Mon, 29 Jul 2013 22:21:50 -0700 Subject: [PATCH 215/221] made SimpleUpdater consistent with other updaters --- mllib/src/main/scala/spark/mllib/optimization/Updater.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index e916a92c33..bf506d2f24 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -41,7 +41,8 @@ abstract class Updater extends Serializable { class SimpleUpdater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { - val normGradient = gradient.mul(stepSize / math.sqrt(iter)) + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) (weightsOld.sub(normGradient), 0) } } From f1cab31b73a7f5464f03b9b24b1338d5bb1496da Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 30 Jul 2013 11:06:47 -0700 Subject: [PATCH 216/221] Removed intermediate set for activeTasks, removed progress bar margin --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 3 ++- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index d6f83157d3..bc1a8ae31a 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -93,7 +93,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, HashSet[TaskInfo]()).size.toString + val activeTasks = listener.executorToTasksActive.get(a.toString).map(l => l.size). + getOrElse(0).toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index e2cf84346a..157418fa79 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -111,7 +111,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) val startWidth = "width: %s%%".format((started.toDouble/total)*100) -
    +
    From 218d7c4ed8fef3fae0da0d3adc1aed65e12c5627 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 30 Jul 2013 12:39:17 -0700 Subject: [PATCH 217/221] Fixed style, lowered height of progress bars --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 157418fa79..a843b5ea2f 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -111,7 +111,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) val startWidth = "width: %s%%".format((started.toDouble/total)*100) -
    +
    From 26144c400f176d74714adc0a7b4168031d868413 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 30 Jul 2013 12:40:41 -0700 Subject: [PATCH 218/221] Fixed wrap style --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index bc1a8ae31a..4be2bfa413 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -93,8 +93,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.get(a.toString).map(l => l.size). - getOrElse(0).toString + val activeTasks = listener.executorToTasksActive.get(a.toString).map(l => l.size) + .getOrElse(0).toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString From e35966ae9a0db3cfed9c46923d939fcc3d4eec4e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 13:28:31 -0700 Subject: [PATCH 219/221] Renamed Classification.scala to ClassificationModel.scala and Regression.scala to RegressionModel.scala --- .../{Classification.scala => ClassificationModel.scala} | 0 .../mllib/regression/{Regression.scala => RegressionModel.scala} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename mllib/src/main/scala/spark/mllib/classification/{Classification.scala => ClassificationModel.scala} (100%) rename mllib/src/main/scala/spark/mllib/regression/{Regression.scala => RegressionModel.scala} (100%) diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/classification/Classification.scala rename to mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/regression/Regression.scala rename to mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala From 47011e685449edfe9f91d8f937a5d23d3b359baf Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 13:58:23 -0700 Subject: [PATCH 220/221] Use a tigher bound in logistic regression unit test's prediction validation. --- .../mllib/classification/LogisticRegressionSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index d3fe58a382..8664263935 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -21,11 +21,12 @@ import scala.util.Random import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers import spark.SparkContext -class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { +class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll with ShouldMatchers { val sc = new SparkContext("local", "test") override def afterAll() { @@ -64,8 +65,8 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => (prediction != expected) }.size - // At least 80% of the predictions should be on. - assert(numOffPredictions < input.length / 5) + // At least 83% of the predictions should be on. + ((input.length - numOffPredictions).toDouble / input.length) should be > 0.83 } // Test if we can correctly learn A, B where Y = logistic(A + B*X) From 366f7735ebe1004acf113df257950d287c50471a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 13:59:32 -0700 Subject: [PATCH 221/221] Minor style cleanup of mllib. --- .../classification/LogisticRegression.scala | 29 ++++++++++--------- .../spark/mllib/classification/SVM.scala | 15 +++++----- .../spark/mllib/optimization/Updater.scala | 10 +++---- .../spark/mllib/recommendation/ALS.scala | 5 ++-- .../scala/spark/mllib/regression/Lasso.scala | 15 +++++----- 5 files changed, 39 insertions(+), 35 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index bf3b05dedb..203aa8fdd4 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -135,8 +135,8 @@ class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBa object LogisticRegressionLocalRandomSGD { /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in * gradient descent are initialized using the initial weights provided. * @@ -155,12 +155,13 @@ object LogisticRegressionLocalRandomSGD { initialWeights: Array[Double]) : LogisticRegressionModel = { - new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) + new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train( + input, initialWeights) } /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. * * @param input RDD of (label, array of features) pairs. @@ -180,9 +181,9 @@ object LogisticRegressionLocalRandomSGD { } /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to update - * the gradient in each iteration. + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using the specified step size. We use the entire data + * set to update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. @@ -200,9 +201,9 @@ object LogisticRegressionLocalRandomSGD { } /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update - * the gradient in each iteration. + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using a step size of 1.0. We use the entire data set + * to update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. @@ -218,12 +219,14 @@ object LogisticRegressionLocalRandomSGD { def main(args: Array[String]) { if (args.length != 5) { - println("Usage: LogisticRegression ") + println("Usage: LogisticRegression " + + " ") System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)).map(yx => (yx._1.toInt, yx._2)) - val model = LogisticRegressionLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LogisticRegressionLocalRandomSGD.train( + data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 15b689e7e0..3a6a12814a 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -53,8 +53,8 @@ class SVMModel( -class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, - var numIters: Int) +class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, + var miniBatchFraction: Double, var numIters: Int) extends Logging { /** @@ -163,7 +163,8 @@ object SVMLocalRandomSGD { initialWeights: Array[Double]) : SVMModel = { - new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train( + input, initialWeights) } /** @@ -190,8 +191,8 @@ object SVMLocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to update - * the gradient in each iteration. + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. @@ -211,8 +212,8 @@ object SVMLocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update - * the gradient in each iteration. + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index bf506d2f24..3ebc1409b6 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -25,7 +25,7 @@ abstract class Updater extends Serializable { * Compute an updated value for weights given the gradient, stepSize and iteration number. * Also returns the regularization value computed using the *updated* weights. * - * @param weightsOlds - Column matrix of size nx1 where n is the number of features. + * @param weightsOld - Column matrix of size nx1 where n is the number of features. * @param gradient - Column matrix of size nx1 where n is the number of features. * @param stepSize - step size across iterations * @param iter - Iteration number @@ -34,8 +34,8 @@ abstract class Updater extends Serializable { * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, * and the second element is the regularization value computed using updated weights. */ - def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): - (DoubleMatrix, Double) + def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, + regParam: Double): (DoubleMatrix, Double) } class SimpleUpdater extends Updater { @@ -64,10 +64,10 @@ class L1Updater extends Updater { val newWeights = weightsOld.sub(normGradient) // Soft thresholding val shrinkageVal = regParam * thisIterStepSize - (0 until newWeights.length).foreach(i => { + (0 until newWeights.length).foreach { i => val wi = newWeights.get(i) newWeights.put(i, signum(wi) * max(0.0, abs(wi) - shrinkageVal)) - }) + } (newWeights, newWeights.norm1 * regParam) } } diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 7da96397a6..7281b2fcb9 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -35,8 +35,7 @@ import org.jblas.{DoubleMatrix, SimpleBlas, Solve} * of the elements within this block, and the list of destination blocks that each user or * product will need to send its feature vector to. */ -private[recommendation] case class OutLinkBlock( - elementIds: Array[Int], shouldSend: Array[BitSet]) +private[recommendation] case class OutLinkBlock(elementIds: Array[Int], shouldSend: Array[BitSet]) /** @@ -105,7 +104,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l } /** - * Run ALS with the configured parmeters on an input RDD of (user, product, rating) triples. + * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. * Returns a MatrixFactorizationModel with feature vectors for each user and product. */ def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index 1952658bb2..e8b1ed8a48 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -53,8 +53,8 @@ class LassoModel( } -class LassoLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, - var numIters: Int) +class LassoLocalRandomSGD private (var stepSize: Double, var regParam: Double, + var miniBatchFraction: Double, var numIters: Int) extends Logging { /** @@ -163,7 +163,8 @@ object LassoLocalRandomSGD { initialWeights: Array[Double]) : LassoModel = { - new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train( + input, initialWeights) } /** @@ -190,8 +191,8 @@ object LassoLocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to update - * the gradient in each iteration. + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. @@ -211,8 +212,8 @@ object LassoLocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update - * the gradient in each iteration. + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run.

    Back to Master