diff --git a/.gitignore b/.gitignore index ad72588b472d6..4f177c82ae5e0 100644 --- a/.gitignore +++ b/.gitignore @@ -7,7 +7,7 @@ sbt/*.jar .settings .cache -.mima-excludes +.generated-mima* /build/ work/ out/ diff --git a/.rat-excludes b/.rat-excludes index 15589702c5599..15344dfb292db 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -3,6 +3,7 @@ target .project .classpath .mima-excludes +.generated-mima-excludes .rat-excludes .*md derby.log @@ -21,6 +22,7 @@ spark-env.sh.template log4j-defaults.properties sorttable.js .*txt +.*json .*data .*log cloudpickle.py diff --git a/README.md b/README.md index 9c2e32b90f162..6211a5889a3f5 100644 --- a/README.md +++ b/README.md @@ -9,13 +9,14 @@ You can find the latest Spark documentation, including a programming guide, on the project webpage at . This README file only contains basic setup instructions. - ## Building Spark Spark is built on Scala 2.10. To build Spark and its example programs, run: ./sbt/sbt assembly +(You do not need to do this if you downloaded a pre-built package.) + ## Interactive Scala Shell The easiest way to start using Spark is through the Scala shell: @@ -41,9 +42,9 @@ And run the following command, which should also return 1000: Spark also comes with several sample programs in the `examples` directory. To run one of them, use `./bin/run-example [params]`. For example: - ./bin/run-example org.apache.spark.examples.SparkLR + ./bin/run-example SparkPi -will run the Logistic Regression example locally. +will run the Pi example locally. You can set the MASTER environment variable when running examples to submit examples to a cluster. This can be a mesos:// or spark:// URL, diff --git a/assembly/pom.xml b/assembly/pom.xml index abd8935339992..0c60b66c3daca 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml @@ -96,7 +96,7 @@ *:* - org.datanucleus:* + org/datanucleus/** META-INF/*.SF META-INF/*.DSA META-INF/*.RSA diff --git a/bagel/pom.xml b/bagel/pom.xml index 355f437c5b16a..c8e39a415af28 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 8e0f82ddb8897..55241d33cd3f0 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -38,8 +38,6 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo sc.stop() sc = null } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") } test("halting by voting") { @@ -82,7 +80,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo test("large number of iterations") { // This tests whether jobs with a large number of iterations finish in a reasonable time, // because non-memoized recursion in RDD or DAGScheduler used to cause them to hang - failAfter(10 seconds) { + failAfter(30 seconds) { sc = new SparkContext("local", "test") val verts = sc.parallelize((1 to 4).map(id => (id.toString, new TestVertex(true, 0)))) val msgs = sc.parallelize(Array[(String, TestMessage)]()) @@ -103,7 +101,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo sc = new SparkContext("local", "test") val verts = sc.parallelize((1 to 4).map(id => (id.toString, new TestVertex(true, 0)))) val msgs = sc.parallelize(Array[(String, TestMessage)]()) - val numSupersteps = 50 + val numSupersteps = 20 val result = Bagel.run(sc, verts, msgs, sc.defaultParallelism, StorageLevel.DISK_ONLY) { (self: TestVertex, msgs: Option[Array[TestMessage]], superstep: Int) => diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 065553eb31939..58710cd1bd548 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -20,6 +20,13 @@ 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. +rem If we're called from spark-class2.cmd, it already set enabledelayedexpansion and setting +rem it here would stop us from affecting its copy of the CLASSPATH variable; otherwise we +rem need to set it here because we use !datanucleus_jars! below. +if "%DONT_PRINT_CLASSPATH%"=="1" goto skip_delayed_expansion +setlocal enabledelayedexpansion +:skip_delayed_expansion + set SCALA_VERSION=2.10 rem Figure out where the Spark framework is installed @@ -31,7 +38,7 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Build up classpath set CLASSPATH=%FWDIR%conf if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( + for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( set ASSEMBLY_JAR=%%d ) ) else ( @@ -42,6 +49,21 @@ if exist "%FWDIR%RELEASE" ( set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% +rem When Hive support is needed, Datanucleus jars must be included on the classpath. +rem Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +rem Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is +rem built with Hive, so look for them there. +if exist "%FWDIR%RELEASE" ( + set datanucleus_dir=%FWDIR%lib +) else ( + set datanucleus_dir=%FWDIR%lib_managed\jars +) +set "datanucleus_jars=" +for %%d in ("%datanucleus_dir%\datanucleus-*.jar") do ( + set datanucleus_jars=!datanucleus_jars!;%%d +) +set CLASSPATH=%CLASSPATH%;%datanucleus_jars% + set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 7df43a555d562..2cf4e381c1c88 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -38,8 +38,10 @@ else JAR_CMD="jar" fi -# First check if we have a dependencies jar. If so, include binary classes with the deps jar -if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then +# A developer option to prepend more recently compiled Spark classes +if [ -n "$SPARK_PREPEND_CLASSES" ]; then + echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\ + "classes ahead of assembly." >&2 CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" @@ -51,17 +53,31 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" +fi - ASSEMBLY_JAR=$(ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar 2>/dev/null) +# Use spark-assembly jar from either RELEASE or assembly directory +if [ -f "$FWDIR/RELEASE" ]; then + assembly_folder="$FWDIR"/lib else - # Else use spark-assembly jar from either RELEASE or assembly directory - if [ -f "$FWDIR/RELEASE" ]; then - ASSEMBLY_JAR=$(ls "$FWDIR"/lib/spark-assembly*hadoop*.jar 2>/dev/null) - else - ASSEMBLY_JAR=$(ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*.jar 2>/dev/null) - fi + assembly_folder="$ASSEMBLY_DIR" fi +num_jars=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l) +if [ "$num_jars" -eq "0" ]; then + echo "Failed to find Spark assembly in $assembly_folder" + echo "You need to build Spark before running this program." + exit 1 +fi +if [ "$num_jars" -gt "1" ]; then + jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar") + echo "Found multiple Spark assembly jars in $assembly_folder:" + echo "$jars_list" + echo "Please remove all but one jar." + exit 1 +fi + +ASSEMBLY_JAR=$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null) + # Verify that versions of java used to build the jars and run Spark are compatible jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then diff --git a/bin/pyspark b/bin/pyspark index 10e35e0f1734e..0b5ed40e2157d 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -17,7 +17,7 @@ # limitations under the License. # -# Figure out where the Scala framework is installed +# Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME @@ -25,6 +25,12 @@ export SPARK_HOME="$FWDIR" SCALA_VERSION=2.10 +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./bin/pyspark [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + # Exit if the user hasn't compiled Spark if [ ! -f "$FWDIR/RELEASE" ]; then # Exit if the user hasn't compiled Spark @@ -39,7 +45,7 @@ fi . $FWDIR/bin/load-spark-env.sh # Figure out which Python executable to use -if [ -z "$PYSPARK_PYTHON" ] ; then +if [[ -z "$PYSPARK_PYTHON" ]]; then PYSPARK_PYTHON="python" fi export PYSPARK_PYTHON @@ -52,13 +58,44 @@ export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH export OLD_PYTHONSTARTUP=$PYTHONSTARTUP export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py -if [ -n "$IPYTHON_OPTS" ]; then +# If IPython options are specified, assume user wants to run IPython +if [[ -n "$IPYTHON_OPTS" ]]; then IPYTHON=1 fi -# Only use ipython if no command line arguments were provided [SPARK-1134] -if [[ "$IPYTHON" = "1" && $# = 0 ]] ; then - exec ipython $IPYTHON_OPTS +# Build up arguments list manually to preserve quotes and backslashes. +# We export Spark submit arguments as an environment variable because shell.py must run as a +# PYTHONSTARTUP script, which does not take in arguments. This is required for IPython notebooks. + +PYSPARK_SUBMIT_ARGS="" +whitespace="[[:space:]]" +for i in "$@"; do + if [[ $i =~ \" ]]; then i=$(echo $i | sed 's/\"/\\\"/g'); fi + if [[ $i =~ $whitespace ]]; then i=\"$i\"; fi + PYSPARK_SUBMIT_ARGS="$PYSPARK_SUBMIT_ARGS $i" +done +export PYSPARK_SUBMIT_ARGS + +# For pyspark tests +if [[ -n "$SPARK_TESTING" ]]; then + if [[ -n "$PYSPARK_DOC_TEST" ]]; then + exec "$PYSPARK_PYTHON" -m doctest $1 + else + exec "$PYSPARK_PYTHON" $1 + fi + exit +fi + +# If a python file is provided, directly run spark-submit. +if [[ "$1" =~ \.py$ ]]; then + echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2 + echo -e "Use ./bin/spark-submit \n" 1>&2 + exec $FWDIR/bin/spark-submit "$@" else - exec "$PYSPARK_PYTHON" "$@" + # Only use ipython if no command line arguments were provided [SPARK-1134] + if [[ "$IPYTHON" = "1" ]]; then + exec ipython $IPYTHON_OPTS + else + exec "$PYSPARK_PYTHON" + fi fi diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index d7cfd5eec501c..0ef9eea95342e 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -31,7 +31,7 @@ set FOUND_JAR=0 for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( set FOUND_JAR=1 ) -if "%FOUND_JAR%"=="0" ( +if [%FOUND_JAR%] == [0] ( echo Failed to find Spark assembly JAR. echo You need to build Spark with sbt\sbt assembly before running this program. goto exit @@ -42,15 +42,30 @@ rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Figure out which Python to use. -if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python +if [%PYSPARK_PYTHON%] == [] set PYSPARK_PYTHON=python set PYTHONPATH=%FWDIR%python;%PYTHONPATH% set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py +set PYSPARK_SUBMIT_ARGS=%* echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH% -"%PYSPARK_PYTHON%" %* +rem Check whether the argument is a file +for /f %%i in ('echo %1^| findstr /R "\.py"') do ( + set PYTHON_FILE=%%i +) + +if [%PYTHON_FILE%] == [] ( + %PYSPARK_PYTHON% +) else ( + echo. + echo WARNING: Running python applications through ./bin/pyspark.cmd is deprecated as of Spark 1.0. + echo Use ./bin/spark-submit ^ + echo. + "%FWDIR%\bin\spark-submit.cmd" %PYSPARK_SUBMIT_ARGS% +) + :exit diff --git a/bin/run-example b/bin/run-example index 146951ac0ee56..e7a5fe3914fbd 100755 --- a/bin/run-example +++ b/bin/run-example @@ -23,6 +23,16 @@ FWDIR="$(cd `dirname $0`/..; pwd)" export SPARK_HOME="$FWDIR" EXAMPLES_DIR="$FWDIR"/examples +if [ -n "$1" ]; then + EXAMPLE_CLASS="$1" + shift +else + echo "Usage: ./bin/run-example [example-args]" + echo " - set MASTER=XX to use a specific master" + echo " - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression)" + exit 1 +fi + if [ -f "$FWDIR/RELEASE" ]; then export SPARK_EXAMPLES_JAR=`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar` elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then @@ -37,23 +47,12 @@ fi EXAMPLE_MASTER=${MASTER:-"local[*]"} -if [ -n "$1" ]; then - EXAMPLE_CLASS="$1" - shift -else - echo "usage: ./bin/run-example [example-args]" - echo " - set MASTER=XX to use a specific master" - echo " - can use abbreviated example class name (e.g. SparkPi, mllib.MovieLensALS)" - echo - exit -1 -fi - if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then EXAMPLE_CLASS="org.apache.spark.examples.$EXAMPLE_CLASS" fi -./bin/spark-submit \ +"$FWDIR"/bin/spark-submit \ --master $EXAMPLE_MASTER \ --class $EXAMPLE_CLASS \ - $SPARK_EXAMPLES_JAR \ + "$SPARK_EXAMPLES_JAR" \ "$@" diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index 40abb9af74246..eadedd7fa61ff 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -30,7 +30,9 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Test that an argument was given if not "x%1"=="x" goto arg_given - echo Usage: run-example ^ [^] + echo Usage: run-example ^ [example-args] + echo - set MASTER=XX to use a specific master + echo - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression) goto exit :arg_given @@ -38,8 +40,14 @@ set EXAMPLES_DIR=%FWDIR%examples rem Figure out the JAR file that our examples were packaged into. set SPARK_EXAMPLES_JAR= -for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*assembly*.jar") do ( - set SPARK_EXAMPLES_JAR=%%d +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%lib\spark-examples*.jar") do ( + set SPARK_EXAMPLES_JAR=%%d + ) +) else ( + for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*.jar") do ( + set SPARK_EXAMPLES_JAR=%%d + ) ) if "x%SPARK_EXAMPLES_JAR%"=="x" ( echo Failed to find Spark examples assembly JAR. @@ -47,15 +55,34 @@ if "x%SPARK_EXAMPLES_JAR%"=="x" ( goto exit ) -rem Compute Spark classpath using external script -set DONT_PRINT_CLASSPATH=1 -call "%FWDIR%bin\compute-classpath.cmd" -set DONT_PRINT_CLASSPATH=0 -set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH% +rem Set master from MASTER environment variable if given +if "x%MASTER%"=="x" ( + set EXAMPLE_MASTER=local[*] +) else ( + set EXAMPLE_MASTER=%MASTER% +) + +rem If the EXAMPLE_CLASS does not start with org.apache.spark.examples, add that +set EXAMPLE_CLASS=%1 +set PREFIX=%EXAMPLE_CLASS:~0,25% +if not %PREFIX%==org.apache.spark.examples ( + set EXAMPLE_CLASS=org.apache.spark.examples.%EXAMPLE_CLASS% +) + +rem Get the tail of the argument list, to skip the first one. This is surprisingly +rem complicated on Windows. +set "ARGS=" +:top +shift +if "%~1" neq "" ( + set ARGS=%ARGS% "%~1" + goto :top +) +if defined ARGS set ARGS=%ARGS:~1% -rem Figure out where java is. -set RUNNER=java -if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java +call "%FWDIR%bin\spark-submit.cmd" ^ + --master %EXAMPLE_MASTER% ^ + --class %EXAMPLE_CLASS% ^ + "%SPARK_EXAMPLES_JAR%" %ARGS% -"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* :exit diff --git a/bin/spark-class b/bin/spark-class index 6480ccb58d6aa..cfe363a71da31 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -24,7 +24,7 @@ esac SCALA_VERSION=2.10 -# Figure out where the Scala framework is installed +# Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME @@ -99,31 +99,14 @@ else fi # Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="$OUR_JAVA_OPTS" +JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" fi export JAVA_OPTS -# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! - -if [ ! -f "$FWDIR/RELEASE" ]; then - # Exit if the user hasn't compiled Spark - num_jars=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar" | wc -l) - jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar") - if [ "$num_jars" -eq "0" ]; then - echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2 - echo "You need to build Spark before running this program." >&2 - exit 1 - fi - if [ "$num_jars" -gt "1" ]; then - echo "Found multiple Spark assembly jars in $FWDIR/assembly/target/scala-$SCALA_VERSION:" >&2 - echo "$jars_list" - echo "Please remove all but one jar." - exit 1 - fi -fi +# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! TOOLS_DIR="$FWDIR"/tools SPARK_TOOLS_JAR="" diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 4302c1b6b7ff4..e420eb409e529 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -17,6 +17,8 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem +setlocal enabledelayedexpansion + set SCALA_VERSION=2.10 rem Figure out where the Spark framework is installed @@ -75,8 +77,8 @@ rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SP ) rem Set JAVA_OPTS to be able to load native libraries and to set heap size -set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% -rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! +set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! rem Test whether the user has built Spark if exist "%FWDIR%RELEASE" goto skip_build_test diff --git a/bin/spark-shell b/bin/spark-shell index 7f03349c5e910..850e9507ec38f 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -28,15 +28,15 @@ esac # Enter posix mode for bash set -o posix -if [[ "$@" == *--help* ]]; then +## Global script variables +FWDIR="$(cd `dirname $0`/..; pwd)" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then echo "Usage: ./bin/spark-shell [options]" - ./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 fi -## Global script variables -FWDIR="$(cd `dirname $0`/..; pwd)" - function main(){ if $cygwin; then # Workaround for issue involving JLine and Cygwin @@ -46,11 +46,11 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main fi } diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index ca0c722c926f5..4b9708a8c03f3 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -19,4 +19,4 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-internal %* --class org.apache.spark.repl.Main +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main diff --git a/bin/spark-submit b/bin/spark-submit index 63903b17a2902..9e7cecedd0325 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -41,5 +41,5 @@ if [ -n "$DRIVER_MEMORY" ] && [ $DEPLOY_MODE == "client" ]; then export SPARK_DRIVER_MEMORY=$DRIVER_MEMORY fi -$SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" +exec $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" diff --git a/core/pom.xml b/core/pom.xml index bab50f5ce2888..bd6767e03bb9d 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml @@ -67,6 +67,12 @@ org.apache.commons commons-lang3 + + org.apache.commons + commons-math3 + 3.3 + test + com.google.code.findbugs jsr305 @@ -235,7 +241,7 @@ org.easymock - easymock + easymockclassextension test @@ -258,35 +264,6 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - org.apache.maven.plugins - maven-antrun-plugin - - - test - - run - - - true - - - - - - - - - - - - - - - - - - org.scalatest scalatest-maven-plugin diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 599c3ac9b57c0..a8bc141208a94 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -87,3 +87,24 @@ span.kill-link { span.kill-link a { color: gray; } + +span.expand-details { + font-size: 10pt; + cursor: pointer; + color: grey; + float: right; +} + +.stage-details { + max-height: 100px; + overflow-y: auto; + margin: 0; + transition: max-height 0.5s ease-out, padding 0.5s ease-out; +} + +.stage-details.collapsed { + max-height: 0; + padding-top: 0; + padding-bottom: 0; + border: none; +} diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 811610c657b62..315ed91f81df3 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -32,10 +32,14 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { private val loading = new HashSet[RDDBlockId]() /** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */ - def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, + def getOrCompute[T]( + rdd: RDD[T], + split: Partition, + context: TaskContext, storageLevel: StorageLevel): Iterator[T] = { + val key = RDDBlockId(rdd.id, split.index) - logDebug("Looking for partition " + key) + logDebug(s"Looking for partition $key") blockManager.get(key) match { case Some(values) => // Partition is already materialized, so just return its values @@ -45,7 +49,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // Mark the split as loading (unless someone else marks it first) loading.synchronized { if (loading.contains(key)) { - logInfo("Another thread is loading %s, waiting for it to finish...".format(key)) + logInfo(s"Another thread is loading $key, waiting for it to finish...") while (loading.contains(key)) { try { loading.wait() @@ -54,7 +58,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { logWarning(s"Got an exception while waiting for another thread to load $key", e) } } - logInfo("Finished waiting for %s".format(key)) + logInfo(s"Finished waiting for $key") /* See whether someone else has successfully loaded it. The main way this would fail * is for the RDD-level cache eviction policy if someone else has loaded the same RDD * partition but we didn't want to make space for it. However, that case is unlikely @@ -64,7 +68,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { case Some(values) => return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) case None => - logInfo("Whoever was loading %s failed; we'll try it ourselves".format(key)) + logInfo(s"Whoever was loading $key failed; we'll try it ourselves") loading.add(key) } } else { @@ -73,7 +77,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { } try { // If we got here, we have to load the split - logInfo("Partition %s not found, computing it".format(key)) + logInfo(s"Partition $key not found, computing it") val computedValues = rdd.computeOrReadCheckpoint(split, context) // Persist the result, so long as the task is not running locally @@ -97,8 +101,8 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { case Some(values) => values.asInstanceOf[Iterator[T]] case None => - logInfo("Failure to store %s".format(key)) - throw new Exception("Block manager failed to return persisted valued") + logInfo(s"Failure to store $key") + throw new SparkException("Block manager failed to return persisted value") } } else { // In this case the RDD is cached to an array buffer. This will save the results diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index e2d2250982daa..bf3c3a6ceb5ef 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -96,7 +96,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Register a ShuffleDependency for cleanup when it is garbage collected. */ - def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _]) { + def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]) { registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) } diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 2c31cc20211ff..c8c194a111aac 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -20,6 +20,7 @@ package org.apache.spark import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleHandle /** * :: DeveloperApi :: @@ -50,19 +51,24 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output - * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to null, + * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None, * the default serializer, as specified by `spark.serializer` config option, will * be used. */ @DeveloperApi -class ShuffleDependency[K, V]( +class ShuffleDependency[K, V, C]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, - val serializer: Serializer = null) + val serializer: Option[Serializer] = None, + val keyOrdering: Option[Ordering[K]] = None, + val aggregator: Option[Aggregator[K, V, C]] = None) extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() + val shuffleHandle: ShuffleHandle = rdd.context.env.shuffleManager.registerShuffle( + shuffleId, rdd.partitions.size, this) + rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index a6e300d345786..0e3750fdde415 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -59,6 +59,13 @@ private[spark] class HttpFileServer(securityManager: SecurityManager) extends Lo } def addFileToDir(file: File, dir: File) : String = { + // Check whether the file is a directory. If it is, throw a more meaningful exception. + // If we don't catch this, Guava throws a very confusing error message: + // java.io.FileNotFoundException: [file] (No such file or directory) + // even though the directory ([file]) exists. + if (file.isDirectory) { + throw new IllegalArgumentException(s"$file cannot be a directory.") + } Files.copy(file, new File(dir, file.getName)) dir + "/" + file.getName } diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 9155159cf6aeb..e7f75481939a8 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -83,11 +83,17 @@ class HashPartitioner(partitions: Int) extends Partitioner { case _ => false } + + override def hashCode: Int = numPartitions } /** * A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly * equal ranges. The ranges are determined by sampling the content of the RDD passed in. + * + * Note that the actual number of partitions created by the RangePartitioner might not be the same + * as the `partitions` parameter, in the case where the number of sampled records is less than + * the value of `partitions`. */ class RangePartitioner[K : Ordering : ClassTag, V]( partitions: Int, @@ -119,7 +125,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } } - def numPartitions = partitions + def numPartitions = rangeBounds.length + 1 private val binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K] @@ -155,4 +161,16 @@ class RangePartitioner[K : Ordering : ClassTag, V]( case _ => false } + + override def hashCode(): Int = { + val prime = 31 + var result = 1 + var i = 0 + while (i < rangeBounds.length) { + result = prime * result + rangeBounds(i).hashCode + i += 1 + } + result = prime * result + ascending.hashCode + result + } } diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 800616622d7bf..8ce4b91cae8ae 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -238,10 +238,10 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { // Check for legacy configs sys.env.get("SPARK_JAVA_OPTS").foreach { value => - val error = + val warning = s""" |SPARK_JAVA_OPTS was detected (set to '$value'). - |This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + |This is deprecated in Spark 1.0+. | |Please instead use: | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application @@ -249,7 +249,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { | - spark.executor.extraJavaOptions to set -X options for executors | - SPARK_DAEMON_JAVA_OPTS to set java options for standalone daemons (master or worker) """.stripMargin - logError(error) + logWarning(warning) for (key <- Seq(executorOptsKey, driverOptsKey)) { if (getOption(key).isDefined) { @@ -262,16 +262,16 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } sys.env.get("SPARK_CLASSPATH").foreach { value => - val error = + val warning = s""" |SPARK_CLASSPATH was detected (set to '$value'). - | This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + |This is deprecated in Spark 1.0+. | |Please instead use: | - ./spark-submit with --driver-class-path to augment the driver classpath | - spark.executor.extraClassPath to augment the executor classpath """.stripMargin - logError(error) + logWarning(warning) for (key <- Seq(executorClasspathKey, driverClassPathKey)) { if (getOption(key).isDefined) { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 634c10c33fc86..0678bdd02110e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -49,7 +49,7 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} +import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -76,8 +76,8 @@ class SparkContext(config: SparkConf) extends Logging { * :: DeveloperApi :: * Alternative constructor for setting preferred locations where Spark will create executors. * - * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Ca - * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] + * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. + * Can be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] * from a list of input files or InputFormats for the application. */ @DeveloperApi @@ -290,6 +290,9 @@ class SparkContext(config: SparkConf) extends Logging { value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { executorEnvs(envKey) = value } + Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => + executorEnvs("SPARK_PREPEND_CLASSES") = v + } // The Mesos scheduler backend relies on this environment variable to set executor memory. // TODO: Set this only in the Mesos scheduler. executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" @@ -297,7 +300,7 @@ class SparkContext(config: SparkConf) extends Logging { // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { - Option(System.getProperty("user.name")).getOrElse(System.getenv("SPARK_USER")) + Option(System.getenv("SPARK_USER")).getOrElse(System.getProperty("user.name")) }.getOrElse { SparkContext.SPARK_UNKNOWN_USER } @@ -431,12 +434,21 @@ class SparkContext(config: SparkConf) extends Logging { // Methods for creating RDDs - /** Distribute a local Scala collection to form an RDD. */ + /** Distribute a local Scala collection to form an RDD. + * + * @note Parallelize acts lazily. If `seq` is a mutable collection and is + * altered after the call to parallelize and before the first action on the + * RDD, the resultant RDD will reflect the modified collection. Pass a copy of + * the argument to avoid this. + */ def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]()) } - /** Distribute a local Scala collection to form an RDD. */ + /** Distribute a local Scala collection to form an RDD. + * + * This method is identical to `parallelize`. + */ def makeRDD[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { parallelize(seq, numSlices) } @@ -455,7 +467,7 @@ class SparkContext(config: SparkConf) extends Logging { */ def textFile(path: String, minPartitions: Int = defaultMinPartitions): RDD[String] = { hadoopFile(path, classOf[TextInputFormat], classOf[LongWritable], classOf[Text], - minPartitions).map(pair => pair._2.toString) + minPartitions).map(pair => pair._2.toString).setName(path) } /** @@ -496,7 +508,7 @@ class SparkContext(config: SparkConf) extends Logging { classOf[String], classOf[String], updateConf, - minPartitions) + minPartitions).setName(path) } /** @@ -551,7 +563,7 @@ class SparkContext(config: SparkConf) extends Logging { inputFormatClass, keyClass, valueClass, - minPartitions) + minPartitions).setName(path) } /** @@ -623,7 +635,7 @@ class SparkContext(config: SparkConf) extends Logging { val job = new NewHadoopJob(conf) NewFileInputFormat.addInputPath(job, new Path(path)) val updatedConf = job.getConfiguration - new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf) + new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf).setName(path) } /** @@ -718,7 +730,7 @@ class SparkContext(config: SparkConf) extends Logging { minPartitions: Int = defaultMinPartitions ): RDD[T] = { sequenceFile(path, classOf[NullWritable], classOf[BytesWritable], minPartitions) - .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes)) + .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes, Utils.getContextOrSparkClassLoader)) } protected[spark] def checkpointFile[T: ClassTag]( @@ -794,7 +806,7 @@ class SparkContext(config: SparkConf) extends Logging { addedFiles(key) = System.currentTimeMillis // Fetch the file locally in case a job is executed using DAGScheduler.runLocally(). - Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf, env.securityManager) + Utils.fetchFile(path, new File(SparkFiles.getRootDirectory()), conf, env.securityManager) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) postEnvironmentUpdate() @@ -823,9 +835,11 @@ class SparkContext(config: SparkConf) extends Logging { } /** + * :: DeveloperApi :: * Return information about what RDDs are cached, if they are in mem or on disk, how much space * they take, etc. */ + @DeveloperApi def getRDDStorageInfo: Array[RDDInfo] = { StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) } @@ -837,8 +851,10 @@ class SparkContext(config: SparkConf) extends Logging { def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap /** + * :: DeveloperApi :: * Return information about blocks stored in all of the slaves */ + @DeveloperApi def getExecutorStorageStatus: Array[StorageStatus] = { env.blockManager.master.getStorageStatus } @@ -932,13 +948,12 @@ class SparkContext(config: SparkConf) extends Logging { try { env.httpFileServer.addJar(new File(fileName)) } catch { - case e: Exception => { + case e: Exception => // For now just log an error but allow to go through so spark examples work. // The spark examples don't really need the jar distributed since its also // the app jar. logError("Error adding jar (" + e + "), was the --addJars option used?") null - } } } else { env.httpFileServer.addJar(new File(uri.getPath)) @@ -1021,9 +1036,11 @@ class SparkContext(config: SparkConf) extends Logging { * Capture the current user callsite and return a formatted version for printing. If the user * has overridden the call site, this will return the user's version. */ - private[spark] def getCallSite(): String = { - val defaultCallSite = Utils.getCallSiteInfo - Option(getLocalProperty("externalCallSite")).getOrElse(defaultCallSite.toString) + private[spark] def getCallSite(): CallSite = { + Option(getLocalProperty("externalCallSite")) match { + case Some(callSite) => CallSite(callSite, long = "") + case None => Utils.getCallSite + } } /** @@ -1043,11 +1060,11 @@ class SparkContext(config: SparkConf) extends Logging { } val callSite = getCallSite val cleanedFunc = clean(func) - logInfo("Starting job: " + callSite) + logInfo("Starting job: " + callSite.short) val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, resultHandler, localProperties.get) - logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") + logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() } @@ -1128,11 +1145,11 @@ class SparkContext(config: SparkConf) extends Logging { evaluator: ApproximateEvaluator[U, R], timeout: Long): PartialResult[R] = { val callSite = getCallSite - logInfo("Starting job: " + callSite) + logInfo("Starting job: " + callSite.short) val start = System.nanoTime val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, localProperties.get) - logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") + logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s") result } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 720151a6b0f84..8dfa8cc4b5b3f 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -34,6 +34,7 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.ConnectionManager import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -56,7 +57,7 @@ class SparkEnv ( val closureSerializer: Serializer, val cacheManager: CacheManager, val mapOutputTracker: MapOutputTracker, - val shuffleFetcher: ShuffleFetcher, + val shuffleManager: ShuffleManager, val broadcastManager: BroadcastManager, val blockManager: BlockManager, val connectionManager: ConnectionManager, @@ -80,7 +81,7 @@ class SparkEnv ( pythonWorkers.foreach { case(key, worker) => worker.stop() } httpFileServer.stop() mapOutputTracker.stop() - shuffleFetcher.stop() + shuffleManager.stop() broadcastManager.stop() blockManager.stop() blockManager.master.stop() @@ -163,13 +164,20 @@ object SparkEnv extends Logging { def instantiateClass[T](propertyName: String, defaultClassName: String): T = { val name = conf.get(propertyName, defaultClassName) val cls = Class.forName(name, true, Utils.getContextOrSparkClassLoader) - // First try with the constructor that takes SparkConf. If we can't find one, - // use a no-arg constructor instead. + // Look for a constructor taking a SparkConf and a boolean isDriver, then one taking just + // SparkConf, then one taking no arguments try { - cls.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[T] + cls.getConstructor(classOf[SparkConf], java.lang.Boolean.TYPE) + .newInstance(conf, new java.lang.Boolean(isDriver)) + .asInstanceOf[T] } catch { case _: NoSuchMethodException => - cls.getConstructor().newInstance().asInstanceOf[T] + try { + cls.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[T] + } catch { + case _: NoSuchMethodException => + cls.getConstructor().newInstance().asInstanceOf[T] + } } } @@ -219,9 +227,6 @@ object SparkEnv extends Logging { val cacheManager = new CacheManager(blockManager) - val shuffleFetcher = instantiateClass[ShuffleFetcher]( - "spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher") - val httpFileServer = new HttpFileServer(securityManager) httpFileServer.initialize() conf.set("spark.fileserver.uri", httpFileServer.serverUri) @@ -242,6 +247,9 @@ object SparkEnv extends Logging { "." } + val shuffleManager = instantiateClass[ShuffleManager]( + "spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager") + // Warn about deprecated spark.cache.class property if (conf.contains("spark.cache.class")) { logWarning("The spark.cache.class property is no longer being used! Specify storage " + @@ -255,7 +263,7 @@ object SparkEnv extends Logging { closureSerializer, cacheManager, mapOutputTracker, - shuffleFetcher, + shuffleManager, broadcastManager, blockManager, connectionManager, diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 4c8f9ed6fbc02..14fa9d8135afe 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -228,6 +228,50 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) : PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) + /** + * Aggregate the values of each key, using given combine functions and a neutral "zero value". + * This function can return a different result type, U, than the type of the values in this RDD, + * V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, + * as in scala.TraversableOnce. The former operation is used for merging values within a + * partition, and the latter is used for merging values between partitions. To avoid memory + * allocation, both of these functions are allowed to modify and return their first argument + * instead of creating a new U. + */ + def aggregateByKey[U](zeroValue: U, partitioner: Partitioner, seqFunc: JFunction2[U, V, U], + combFunc: JFunction2[U, U, U]): JavaPairRDD[K, U] = { + implicit val ctag: ClassTag[U] = fakeClassTag + fromRDD(rdd.aggregateByKey(zeroValue, partitioner)(seqFunc, combFunc)) + } + + /** + * Aggregate the values of each key, using given combine functions and a neutral "zero value". + * This function can return a different result type, U, than the type of the values in this RDD, + * V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, + * as in scala.TraversableOnce. The former operation is used for merging values within a + * partition, and the latter is used for merging values between partitions. To avoid memory + * allocation, both of these functions are allowed to modify and return their first argument + * instead of creating a new U. + */ + def aggregateByKey[U](zeroValue: U, numPartitions: Int, seqFunc: JFunction2[U, V, U], + combFunc: JFunction2[U, U, U]): JavaPairRDD[K, U] = { + implicit val ctag: ClassTag[U] = fakeClassTag + fromRDD(rdd.aggregateByKey(zeroValue, numPartitions)(seqFunc, combFunc)) + } + + /** + * Aggregate the values of each key, using given combine functions and a neutral "zero value". + * This function can return a different result type, U, than the type of the values in this RDD, + * V. Thus, we need one operation for merging a V into a U and one operation for merging two U's. + * The former operation is used for merging values within a partition, and the latter is used for + * merging values between partitions. To avoid memory allocation, both of these functions are + * allowed to modify and return their first argument instead of creating a new U. + */ + def aggregateByKey[U](zeroValue: U, seqFunc: JFunction2[U, V, U], combFunc: JFunction2[U, U, U]): + JavaPairRDD[K, U] = { + implicit val ctag: ClassTag[U] = fakeClassTag + fromRDD(rdd.aggregateByKey(zeroValue)(seqFunc, combFunc)) + } + /** * Merge the values for each key using an associative function and a neutral "zero value" which * may be added to the result an arbitrary number of times, and must not change the result @@ -672,38 +716,47 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. Uses the provided - * Partitioner to partition the output RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param partitioner partitioner of the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaRDD[(K, Long)] = { - rdd.countApproxDistinctByKey(relativeSD, partitioner) + def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaPairRDD[K, Long] = + { + fromRDD(rdd.countApproxDistinctByKey(relativeSD, partitioner)) } /** - * Return approximate number of distinct values for each key this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. The default value of - * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism - * level. + * Return approximate number of distinct values for each key in this RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param numPartitions number of partitions of the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double = 0.05): JavaRDD[(K, Long)] = { - rdd.countApproxDistinctByKey(relativeSD) + def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaPairRDD[K, Long] = { + fromRDD(rdd.countApproxDistinctByKey(relativeSD, numPartitions)) } - /** * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. HashPartitions the - * output RDD into numPartitions. * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. */ - def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaRDD[(K, Long)] = { - rdd.countApproxDistinctByKey(relativeSD, numPartitions) + def countApproxDistinctByKey(relativeSD: Double): JavaPairRDD[K, Long] = { + fromRDD(rdd.countApproxDistinctByKey(relativeSD)) } /** Assign a name to this RDD */ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index dc698dea75e43..86fb374bef1e3 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -17,10 +17,13 @@ package org.apache.spark.api.java +import java.util.Comparator + import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark._ +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -108,6 +111,28 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) + + /** + * Randomly splits this RDD with the provided weights. + * + * @param weights weights for splits, will be normalized if they don't sum to 1 + * + * @return split RDDs in an array + */ + def randomSplit(weights: Array[Double]): Array[JavaRDD[T]] = + randomSplit(weights, Utils.random.nextLong) + + /** + * Randomly splits this RDD with the provided weights. + * + * @param weights weights for splits, will be normalized if they don't sum to 1 + * @param seed random seed + * + * @return split RDDs in an array + */ + def randomSplit(weights: Array[Double], seed: Long): Array[JavaRDD[T]] = + rdd.randomSplit(weights, seed).map(wrapRDD) + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). @@ -150,6 +175,19 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) rdd.setName(name) this } + + /** + * Return this RDD sorted by the given key function. + */ + def sortBy[S](f: JFunction[T, S], ascending: Boolean, numPartitions: Int): JavaRDD[T] = { + import scala.collection.JavaConverters._ + def fn = (x: T) => f.call(x) + import com.google.common.collect.Ordering // shadows scala.math.Ordering + implicit val ordering = Ordering.natural().asInstanceOf[Ordering[S]] + implicit val ctag: ClassTag[S] = fakeClassTag + wrapRDD(rdd.sortBy(fn, ascending, numPartitions)) + } + } object JavaRDD { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 619bfd75be8eb..330569a8d8837 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -560,12 +560,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return approximate number of distinct elements in the RDD. * - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. The default value of - * relativeSD is 0.05. + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. */ - def countApproxDistinct(relativeSD: Double = 0.05): Long = rdd.countApproxDistinct(relativeSD) + def countApproxDistinct(relativeSD: Double): Long = rdd.countApproxDistinct(relativeSD) def name(): String = rdd.name diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index a7cfee6d01711..1e0493c4855e0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -41,6 +41,12 @@ import org.apache.spark.rdd.RDD * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. */ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround { + /** + * Create a JavaSparkContext that loads settings from system properties (for instance, when + * launching with ./bin/spark-submit). + */ + def this() = this(new SparkContext()) + /** * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters */ diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala new file mode 100644 index 0000000000000..adaa1ef6cf9ff --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.python + +import org.apache.spark.rdd.RDD +import org.apache.spark.Logging +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.io._ +import scala.util.{Failure, Success, Try} +import org.apache.spark.annotation.Experimental + + +/** + * :: Experimental :: + * A trait for use with reading custom classes in PySpark. Implement this trait and add custom + * transformation code by overriding the convert method. + */ +@Experimental +trait Converter[T, U] extends Serializable { + def convert(obj: T): U +} + +private[python] object Converter extends Logging { + + def getInstance(converterClass: Option[String]): Converter[Any, Any] = { + converterClass.map { cc => + Try { + val c = Class.forName(cc).newInstance().asInstanceOf[Converter[Any, Any]] + logInfo(s"Loaded converter: $cc") + c + } match { + case Success(c) => c + case Failure(err) => + logError(s"Failed to load converter: $cc") + throw err + } + }.getOrElse { new DefaultConverter } + } +} + +/** + * A converter that handles conversion of common [[org.apache.hadoop.io.Writable]] objects. + * Other objects are passed through without conversion. + */ +private[python] class DefaultConverter extends Converter[Any, Any] { + + /** + * Converts a [[org.apache.hadoop.io.Writable]] to the underlying primitive, String or + * object representation + */ + private def convertWritable(writable: Writable): Any = { + import collection.JavaConversions._ + writable match { + case iw: IntWritable => iw.get() + case dw: DoubleWritable => dw.get() + case lw: LongWritable => lw.get() + case fw: FloatWritable => fw.get() + case t: Text => t.toString + case bw: BooleanWritable => bw.get() + case byw: BytesWritable => byw.getBytes + case n: NullWritable => null + case aw: ArrayWritable => aw.get().map(convertWritable(_)) + case mw: MapWritable => mapAsJavaMap(mw.map { case (k, v) => + (convertWritable(k), convertWritable(v)) + }.toMap) + case other => other + } + } + + def convert(obj: Any): Any = { + obj match { + case writable: Writable => + convertWritable(writable) + case _ => + obj + } + } +} + +/** Utilities for working with Python objects <-> Hadoop-related objects */ +private[python] object PythonHadoopUtil { + + /** + * Convert a [[java.util.Map]] of properties to a [[org.apache.hadoop.conf.Configuration]] + */ + def mapToConf(map: java.util.Map[String, String]): Configuration = { + import collection.JavaConversions._ + val conf = new Configuration() + map.foreach{ case (k, v) => conf.set(k, v) } + conf + } + + /** + * Merges two configurations, returns a copy of left with keys from right overwriting + * any matching keys in left + */ + def mergeConfs(left: Configuration, right: Configuration): Configuration = { + import collection.JavaConversions._ + val copy = new Configuration(left) + right.iterator().foreach(entry => copy.set(entry.getKey, entry.getValue)) + copy + } + + /** + * Converts an RDD of key-value pairs, where key and/or value could be instances of + * [[org.apache.hadoop.io.Writable]], into an RDD[(K, V)] + */ + def convertRDD[K, V](rdd: RDD[(K, V)], + keyConverter: Converter[Any, Any], + valueConverter: Converter[Any, Any]): RDD[(Any, Any)] = { + rdd.map { case (k, v) => (keyConverter.convert(k), valueConverter.convert(v)) } + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala index 95bec5030bfdd..e230d222b8604 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala @@ -50,4 +50,6 @@ private[spark] class PythonPartitioner( case _ => false } + + override def hashCode: Int = 31 * numPartitions + pyPartitionFunctionId.hashCode } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 57b28b9972366..f6570d335757a 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -28,6 +28,9 @@ import scala.util.Try import net.razorvine.pickle.{Pickler, Unpickler} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.{InputFormat, JobConf} +import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast @@ -266,9 +269,29 @@ private object SpecialLengths { val TIMING_DATA = -3 } -private[spark] object PythonRDD { +private[spark] object PythonRDD extends Logging { val UTF8 = Charset.forName("UTF-8") + /** + * Adapter for calling SparkContext#runJob from Python. + * + * This method will return an iterator of an array that contains all elements in the RDD + * (effectively a collect()), but allows you to run on a certain subset of partitions, + * or to enable local execution. + */ + def runJob( + sc: SparkContext, + rdd: JavaRDD[Array[Byte]], + partitions: JArrayList[Int], + allowLocal: Boolean): Iterator[Array[Byte]] = { + type ByteArray = Array[Byte] + type UnrolledPartition = Array[ByteArray] + val allPartitions: Array[UnrolledPartition] = + sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions, allowLocal) + val flattenedPartition: UnrolledPartition = Array.concat(allPartitions: _*) + flattenedPartition.iterator + } + def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { val file = new DataInputStream(new FileInputStream(filename)) @@ -326,6 +349,180 @@ private[spark] object PythonRDD { } } + /** + * Create an RDD from a path using [[org.apache.hadoop.mapred.SequenceFileInputFormat]], + * key and value class. + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def sequenceFile[K, V]( + sc: JavaSparkContext, + path: String, + keyClassMaybeNull: String, + valueClassMaybeNull: String, + keyConverterClass: String, + valueConverterClass: String, + minSplits: Int) = { + val keyClass = Option(keyClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") + val valueClass = Option(valueClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") + implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]] + implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]] + val kc = kcm.runtimeClass.asInstanceOf[Class[K]] + val vc = vcm.runtimeClass.asInstanceOf[Class[V]] + + val rdd = sc.sc.sequenceFile[K, V](path, kc, vc, minSplits) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + /** + * Create an RDD from a file path, using an arbitrary [[org.apache.hadoop.mapreduce.InputFormat]], + * key and value class. + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]]( + sc: JavaSparkContext, + path: String, + inputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val baseConf = sc.hadoopConfiguration() + val mergedConf = PythonHadoopUtil.mergeConfs(baseConf, conf) + val rdd = + newAPIHadoopRDDFromClassNames[K, V, F](sc, + Some(path), inputFormatClass, keyClass, valueClass, mergedConf) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + /** + * Create an RDD from a [[org.apache.hadoop.conf.Configuration]] converted from a map that is + * passed in from Python, using an arbitrary [[org.apache.hadoop.mapreduce.InputFormat]], + * key and value class. + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]]( + sc: JavaSparkContext, + inputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val rdd = + newAPIHadoopRDDFromClassNames[K, V, F](sc, + None, inputFormatClass, keyClass, valueClass, conf) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + private def newAPIHadoopRDDFromClassNames[K, V, F <: NewInputFormat[K, V]]( + sc: JavaSparkContext, + path: Option[String] = None, + inputFormatClass: String, + keyClass: String, + valueClass: String, + conf: Configuration) = { + implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]] + implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]] + implicit val fcm = ClassTag(Class.forName(inputFormatClass)).asInstanceOf[ClassTag[F]] + val kc = kcm.runtimeClass.asInstanceOf[Class[K]] + val vc = vcm.runtimeClass.asInstanceOf[Class[V]] + val fc = fcm.runtimeClass.asInstanceOf[Class[F]] + val rdd = if (path.isDefined) { + sc.sc.newAPIHadoopFile[K, V, F](path.get, fc, kc, vc, conf) + } else { + sc.sc.newAPIHadoopRDD[K, V, F](conf, fc, kc, vc) + } + rdd + } + + /** + * Create an RDD from a file path, using an arbitrary [[org.apache.hadoop.mapred.InputFormat]], + * key and value class. + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def hadoopFile[K, V, F <: InputFormat[K, V]]( + sc: JavaSparkContext, + path: String, + inputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val baseConf = sc.hadoopConfiguration() + val mergedConf = PythonHadoopUtil.mergeConfs(baseConf, conf) + val rdd = + hadoopRDDFromClassNames[K, V, F](sc, + Some(path), inputFormatClass, keyClass, valueClass, mergedConf) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + /** + * Create an RDD from a [[org.apache.hadoop.conf.Configuration]] converted from a map + * that is passed in from Python, using an arbitrary [[org.apache.hadoop.mapred.InputFormat]], + * key and value class + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def hadoopRDD[K, V, F <: InputFormat[K, V]]( + sc: JavaSparkContext, + inputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val rdd = + hadoopRDDFromClassNames[K, V, F](sc, + None, inputFormatClass, keyClass, valueClass, conf) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + private def hadoopRDDFromClassNames[K, V, F <: InputFormat[K, V]]( + sc: JavaSparkContext, + path: Option[String] = None, + inputFormatClass: String, + keyClass: String, + valueClass: String, + conf: Configuration) = { + implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]] + implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]] + implicit val fcm = ClassTag(Class.forName(inputFormatClass)).asInstanceOf[ClassTag[F]] + val kc = kcm.runtimeClass.asInstanceOf[Class[K]] + val vc = vcm.runtimeClass.asInstanceOf[Class[V]] + val fc = fcm.runtimeClass.asInstanceOf[Class[F]] + val rdd = if (path.isDefined) { + sc.sc.hadoopFile(path.get, fc, kc, vc) + } else { + sc.sc.hadoopRDD(new JobConf(conf), fc, kc, vc) + } + rdd + } + def writeUTF(str: String, dataOut: DataOutputStream) { val bytes = str.getBytes(UTF8) dataOut.writeInt(bytes.length) diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala new file mode 100644 index 0000000000000..9a012e7254901 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.python + +import scala.util.Try +import org.apache.spark.rdd.RDD +import org.apache.spark.Logging +import scala.util.Success +import scala.util.Failure +import net.razorvine.pickle.Pickler + + +/** Utilities for serialization / deserialization between Python and Java, using Pickle. */ +private[python] object SerDeUtil extends Logging { + + private def checkPickle(t: (Any, Any)): (Boolean, Boolean) = { + val pickle = new Pickler + val kt = Try { + pickle.dumps(t._1) + } + val vt = Try { + pickle.dumps(t._2) + } + (kt, vt) match { + case (Failure(kf), Failure(vf)) => + logWarning(s""" + |Failed to pickle Java object as key: ${t._1.getClass.getSimpleName}, falling back + |to 'toString'. Error: ${kf.getMessage}""".stripMargin) + logWarning(s""" + |Failed to pickle Java object as value: ${t._2.getClass.getSimpleName}, falling back + |to 'toString'. Error: ${vf.getMessage}""".stripMargin) + (true, true) + case (Failure(kf), _) => + logWarning(s""" + |Failed to pickle Java object as key: ${t._1.getClass.getSimpleName}, falling back + |to 'toString'. Error: ${kf.getMessage}""".stripMargin) + (true, false) + case (_, Failure(vf)) => + logWarning(s""" + |Failed to pickle Java object as value: ${t._2.getClass.getSimpleName}, falling back + |to 'toString'. Error: ${vf.getMessage}""".stripMargin) + (false, true) + case _ => + (false, false) + } + } + + /** + * Convert an RDD of key-value pairs to an RDD of serialized Python objects, that is usable + * by PySpark. By default, if serialization fails, toString is called and the string + * representation is serialized + */ + def rddToPython(rdd: RDD[(Any, Any)]): RDD[Array[Byte]] = { + val (keyFailed, valueFailed) = checkPickle(rdd.first()) + rdd.mapPartitions { iter => + val pickle = new Pickler + iter.map { case (k, v) => + if (keyFailed && valueFailed) { + pickle.dumps(Array(k.toString, v.toString)) + } else if (keyFailed) { + pickle.dumps(Array(k.toString, v)) + } else if (!keyFailed && valueFailed) { + pickle.dumps(Array(k, v.toString)) + } else { + pickle.dumps(Array(k, v)) + } + } + } + } + +} + diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala new file mode 100644 index 0000000000000..f0e3fb9aff5a0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.python + +import org.apache.spark.SparkContext +import org.apache.hadoop.io._ +import scala.Array +import java.io.{DataOutput, DataInput} +import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat +import org.apache.spark.api.java.JavaSparkContext + +/** + * A class to test MsgPack serialization on the Scala side, that will be deserialized + * in Python + * @param str + * @param int + * @param double + */ +case class TestWritable(var str: String, var int: Int, var double: Double) extends Writable { + def this() = this("", 0, 0.0) + + def getStr = str + def setStr(str: String) { this.str = str } + def getInt = int + def setInt(int: Int) { this.int = int } + def getDouble = double + def setDouble(double: Double) { this.double = double } + + def write(out: DataOutput) = { + out.writeUTF(str) + out.writeInt(int) + out.writeDouble(double) + } + + def readFields(in: DataInput) = { + str = in.readUTF() + int = in.readInt() + double = in.readDouble() + } +} + +class TestConverter extends Converter[Any, Any] { + import collection.JavaConversions._ + override def convert(obj: Any) = { + val m = obj.asInstanceOf[MapWritable] + seqAsJavaList(m.keySet.map(w => w.asInstanceOf[DoubleWritable].get()).toSeq) + } +} + +/** + * This object contains method to generate SequenceFile test data and write it to a + * given directory (probably a temp directory) + */ +object WriteInputFormatTestDataGenerator { + import SparkContext._ + + def main(args: Array[String]) { + val path = args(0) + val sc = new JavaSparkContext("local[4]", "test-writables") + generateData(path, sc) + } + + def generateData(path: String, jsc: JavaSparkContext) { + val sc = jsc.sc + + val basePath = s"$path/sftestdata/" + val textPath = s"$basePath/sftext/" + val intPath = s"$basePath/sfint/" + val doublePath = s"$basePath/sfdouble/" + val arrPath = s"$basePath/sfarray/" + val mapPath = s"$basePath/sfmap/" + val classPath = s"$basePath/sfclass/" + val bytesPath = s"$basePath/sfbytes/" + val boolPath = s"$basePath/sfbool/" + val nullPath = s"$basePath/sfnull/" + + /* + * Create test data for IntWritable, DoubleWritable, Text, BytesWritable, + * BooleanWritable and NullWritable + */ + val intKeys = Seq((1, "aa"), (2, "bb"), (2, "aa"), (3, "cc"), (2, "bb"), (1, "aa")) + sc.parallelize(intKeys).saveAsSequenceFile(intPath) + sc.parallelize(intKeys.map{ case (k, v) => (k.toDouble, v) }).saveAsSequenceFile(doublePath) + sc.parallelize(intKeys.map{ case (k, v) => (k.toString, v) }).saveAsSequenceFile(textPath) + sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes) }).saveAsSequenceFile(bytesPath) + val bools = Seq((1, true), (2, true), (2, false), (3, true), (2, false), (1, false)) + sc.parallelize(bools).saveAsSequenceFile(boolPath) + sc.parallelize(intKeys).map{ case (k, v) => + (new IntWritable(k), NullWritable.get()) + }.saveAsSequenceFile(nullPath) + + // Create test data for ArrayWritable + val data = Seq( + (1, Array(1.0, 2.0, 3.0)), + (2, Array(3.0, 4.0, 5.0)), + (3, Array(4.0, 5.0, 6.0)) + ) + sc.parallelize(data, numSlices = 2) + .map{ case (k, v) => + (new IntWritable(k), new ArrayWritable(classOf[DoubleWritable], v.map(new DoubleWritable(_)))) + }.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, ArrayWritable]](arrPath) + + // Create test data for MapWritable, with keys DoubleWritable and values Text + val mapData = Seq( + (1, Map(2.0 -> "aa")), + (2, Map(3.0 -> "bb")), + (2, Map(1.0 -> "cc")), + (3, Map(2.0 -> "dd")), + (2, Map(1.0 -> "aa")), + (1, Map(3.0 -> "bb")) + ) + sc.parallelize(mapData, numSlices = 2).map{ case (i, m) => + val mw = new MapWritable() + val k = m.keys.head + val v = m.values.head + mw.put(new DoubleWritable(k), new Text(v)) + (new IntWritable(i), mw) + }.saveAsSequenceFile(mapPath) + + // Create test data for arbitrary custom writable TestWritable + val testClass = Seq( + ("1", TestWritable("test1", 123, 54.0)), + ("2", TestWritable("test2", 456, 8762.3)), + ("1", TestWritable("test3", 123, 423.1)), + ("3", TestWritable("test56", 456, 423.5)), + ("2", TestWritable("test2", 123, 5435.2)) + ) + val rdd = sc.parallelize(testClass, numSlices = 2).map{ case (k, v) => (new Text(k), v) } + rdd.saveAsNewAPIHadoopFile(classPath, + classOf[Text], classOf[TestWritable], + classOf[SequenceFileOutputFormat[Text, TestWritable]]) + } + + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index aeb159adc31d9..c371dc3a51c73 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -81,7 +81,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends case "kill" => val driverId = driverArgs.driverId - val killFuture = masterActor ! RequestKillDriver(driverId) + masterActor ! RequestKillDriver(driverId) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 37dfa7fec0831..9f34d01e6db48 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -19,9 +19,9 @@ package org.apache.spark.deploy private[spark] object ExecutorState extends Enumeration { - val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value + val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST, EXITED = Value type ExecutorState = Value - def isFinished(state: ExecutorState): Boolean = Seq(KILLED, FAILED, LOST).contains(state) + def isFinished(state: ExecutorState): Boolean = Seq(KILLED, FAILED, LOST, EXITED).contains(state) } diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index e20d4486c8f0c..0d6751f3fa6d2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -17,10 +17,13 @@ package org.apache.spark.deploy +import java.net.URI + import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import org.apache.spark.api.python.{PythonUtils, RedirectThread} +import org.apache.spark.util.Utils /** * A main class used by spark-submit to launch Python applications. It executes python as a @@ -28,12 +31,15 @@ import org.apache.spark.api.python.{PythonUtils, RedirectThread} */ object PythonRunner { def main(args: Array[String]) { - val primaryResource = args(0) + val pythonFile = args(0) val pyFiles = args(1) val otherArgs = args.slice(2, args.length) - val pythonExec = sys.env.get("PYSPARK_PYTHON").getOrElse("python") // TODO: get this from conf + // Format python file paths before adding them to the PYTHONPATH + val formattedPythonFile = formatPath(pythonFile) + val formattedPyFiles = formatPaths(pyFiles) + // Launch a Py4J gateway server for the process to connect to; this will let it see our // Java system properties and such val gatewayServer = new py4j.GatewayServer(null, 0) @@ -42,13 +48,13 @@ object PythonRunner { // Build up a PYTHONPATH that includes the Spark assembly JAR (where this class is), the // python directories in SPARK_HOME (if set), and any files in the pyFiles argument val pathElements = new ArrayBuffer[String] - pathElements ++= pyFiles.split(",") + pathElements ++= formattedPyFiles pathElements += PythonUtils.sparkPythonPath pathElements += sys.env.getOrElse("PYTHONPATH", "") val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) // Launch Python process - val builder = new ProcessBuilder(Seq(pythonExec, "-u", primaryResource) ++ otherArgs) + val builder = new ProcessBuilder(Seq(pythonExec, "-u", formattedPythonFile) ++ otherArgs) val env = builder.environment() env.put("PYTHONPATH", pythonPath) env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort) @@ -59,4 +65,50 @@ object PythonRunner { System.exit(process.waitFor()) } + + /** + * Format the python file path so that it can be added to the PYTHONPATH correctly. + * + * Python does not understand URI schemes in paths. Before adding python files to the + * PYTHONPATH, we need to extract the path from the URI. This is safe to do because we + * currently only support local python files. + */ + def formatPath(path: String, testWindows: Boolean = false): String = { + if (Utils.nonLocalPaths(path, testWindows).nonEmpty) { + throw new IllegalArgumentException("Launching Python applications through " + + s"spark-submit is currently only supported for local files: $path") + } + val windows = Utils.isWindows || testWindows + var formattedPath = if (windows) Utils.formatWindowsPath(path) else path + + // Strip the URI scheme from the path + formattedPath = + new URI(formattedPath).getScheme match { + case Utils.windowsDrive(d) if windows => formattedPath + case null => formattedPath + case _ => new URI(formattedPath).getPath + } + + // Guard against malformed paths potentially throwing NPE + if (formattedPath == null) { + throw new IllegalArgumentException(s"Python file path is malformed: $path") + } + + // In Windows, the drive should not be prefixed with "/" + // For instance, python does not understand "/C:/path/to/sheep.py" + formattedPath = if (windows) formattedPath.stripPrefix("/") else formattedPath + formattedPath + } + + /** + * Format each python file path in the comma-delimited list of paths, so it can be + * added to the PYTHONPATH correctly. + */ + def formatPaths(paths: String, testWindows: Boolean = false): Array[String] = { + Option(paths).getOrElse("") + .split(",") + .filter(_.nonEmpty) + .map { p => formatPath(p, testWindows) } + } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index e86182e4c56ce..7e9a9344e61f9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -41,10 +41,10 @@ object SparkSubmit { private var clusterManager: Int = LOCAL /** - * A special jar name that indicates the class being run is inside of Spark itself, - * and therefore no user jar is needed. + * Special primary resource names that represent shells rather than application jars. */ - private val RESERVED_JAR_NAME = "spark-internal" + private val SPARK_SHELL = "spark-shell" + private val PYSPARK_SHELL = "pyspark-shell" def main(args: Array[String]) { val appArgs = new SparkSubmitArguments(args) @@ -71,8 +71,8 @@ object SparkSubmit { * entries for the child, a list of system properties, a list of env vars * and the main class for the child */ - private[spark] def createLaunchEnv(args: SparkSubmitArguments): (ArrayBuffer[String], - ArrayBuffer[String], Map[String, String], String) = { + private[spark] def createLaunchEnv(args: SparkSubmitArguments) + : (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { if (args.master.startsWith("local")) { clusterManager = LOCAL } else if (args.master.startsWith("yarn")) { @@ -121,24 +121,30 @@ object SparkSubmit { printErrorAndExit("Cannot currently run driver on the cluster in Mesos") } - // If we're running a Python app, set the Java class to run to be our PythonRunner, add - // Python files to deployment list, and pass the main file and Python path to PythonRunner + // If we're running a python app, set the main class to our specific python runner if (isPython) { if (deployOnCluster) { printErrorAndExit("Cannot currently run Python driver programs on cluster") } - args.mainClass = "org.apache.spark.deploy.PythonRunner" - args.files = mergeFileLists(args.files, args.pyFiles, args.primaryResource) - val pyFiles = Option(args.pyFiles).getOrElse("") - args.childArgs = ArrayBuffer(args.primaryResource, pyFiles) ++ args.childArgs - args.primaryResource = RESERVED_JAR_NAME - sysProps("spark.submit.pyFiles") = pyFiles + if (args.primaryResource == PYSPARK_SHELL) { + args.mainClass = "py4j.GatewayServer" + args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0") + } else { + // If a python file is provided, add it to the child arguments and list of files to deploy. + // Usage: PythonAppRunner
[app arguments] + args.mainClass = "org.apache.spark.deploy.PythonRunner" + args.childArgs = ArrayBuffer(args.primaryResource, args.pyFiles) ++ args.childArgs + args.files = mergeFileLists(args.files, args.primaryResource) + } + args.files = mergeFileLists(args.files, args.pyFiles) + // Format python file paths properly before adding them to the PYTHONPATH + sysProps("spark.submit.pyFiles") = PythonRunner.formatPaths(args.pyFiles).mkString(",") } // If we're deploying into YARN, use yarn.Client as a wrapper around the user class if (!deployOnCluster) { childMainClass = args.mainClass - if (args.primaryResource != RESERVED_JAR_NAME) { + if (isUserJar(args.primaryResource)) { childClasspath += args.primaryResource } } else if (clusterManager == YARN) { @@ -219,7 +225,7 @@ object SparkSubmit { // For python files, the primary resource is already distributed as a regular file if (!isYarnCluster && !isPython) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) - if (args.primaryResource != RESERVED_JAR_NAME) { + if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) } sysProps.put("spark.jars", jars.mkString(",")) @@ -293,13 +299,39 @@ object SparkSubmit { } private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { - val localJarFile = new File(new URI(localJar).getPath()) - if (!localJarFile.exists()) { - printWarning(s"Jar $localJar does not exist, skipping.") + val uri = Utils.resolveURI(localJar) + uri.getScheme match { + case "file" | "local" => + val file = new File(uri.getPath) + if (file.exists()) { + loader.addURL(file.toURI.toURL) + } else { + printWarning(s"Local jar $file does not exist, skipping.") + } + case _ => + printWarning(s"Skip remote jar $uri.") } + } + + /** + * Return whether the given primary resource represents a user jar. + */ + private def isUserJar(primaryResource: String): Boolean = { + !isShell(primaryResource) && !isPython(primaryResource) + } - val url = localJarFile.getAbsoluteFile.toURI.toURL - loader.addURL(url) + /** + * Return whether the given primary resource represents a shell. + */ + private[spark] def isShell(primaryResource: String): Boolean = { + primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL + } + + /** + * Return whether the given primary resource requires running python. + */ + private[spark] def isPython(primaryResource: String): Boolean = { + primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 2d327aa3fb27f..f1032ea8dbada 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -118,7 +118,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") } catch { case e: Exception => - SparkSubmit.printErrorAndExit("Failed to read JAR: " + primaryResource) + SparkSubmit.printErrorAndExit("Cannot load main class from JAR: " + primaryResource) return } } @@ -148,6 +148,18 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { SparkSubmit.printErrorAndExit("--py-files given but primary resource is not a Python script") } + // Require all python files to be local, so we can add them to the PYTHONPATH + if (isPython) { + if (Utils.nonLocalPaths(primaryResource).nonEmpty) { + SparkSubmit.printErrorAndExit(s"Only local python files are supported: $primaryResource") + } + val nonLocalPyFiles = Utils.nonLocalPaths(pyFiles).mkString(",") + if (nonLocalPyFiles.nonEmpty) { + SparkSubmit.printErrorAndExit( + s"Only local additional python files are supported: $nonLocalPyFiles") + } + } + if (master.startsWith("yarn")) { val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") if (!hasHadoopEnv && !Utils.isTesting) { @@ -263,19 +275,19 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { parse(tail) case ("--files") :: value :: tail => - files = value + files = Utils.resolveURIs(value) parse(tail) case ("--py-files") :: value :: tail => - pyFiles = value + pyFiles = Utils.resolveURIs(value) parse(tail) case ("--archives") :: value :: tail => - archives = value + archives = Utils.resolveURIs(value) parse(tail) case ("--jars") :: value :: tail => - jars = value + jars = Utils.resolveURIs(value) parse(tail) case ("--help" | "-h") :: tail => @@ -296,13 +308,20 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { val errMessage = s"Unrecognized option '$value'." SparkSubmit.printErrorAndExit(errMessage) case v => - primaryResource = v + primaryResource = + if (!SparkSubmit.isShell(v)) { + Utils.resolveURI(v).toString + } else { + v + } inSparkOpts = false - isPython = v.endsWith(".py") + isPython = SparkSubmit.isPython(v) parse(tail) } } else { - childArgs += value + if (!value.isEmpty) { + childArgs += value + } parse(tail) } @@ -324,10 +343,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | --class CLASS_NAME Your application's main class (for Java / Scala apps). | --name NAME A name of your application. | --jars JARS Comma-separated list of local jars to include on the driver - | and executor classpaths. Doesn't work for drivers in - | standalone mode with "cluster" deploy mode. - | --py-files PY_FILES Comma-separated list of .zip or .egg files to place on the - | PYTHONPATH for Python apps. + | and executor classpaths. + | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to place + | on the PYTHONPATH for Python apps. | --files FILES Comma-separated list of files to be placed in the working | directory of each executor. | --properties-file FILE Path to a file from which to load extra properties. If not @@ -342,6 +360,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | + | --help, -h Show this help message and exit + | --verbose, -v Print additional debug output + | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). | --supervise If given, restarts the driver on failure. @@ -363,16 +384,19 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { object SparkSubmitArguments { /** Load properties present in the given file. */ def getPropertiesFromFile(file: File): Seq[(String, String)] = { - require(file.exists(), s"Properties file ${file.getName} does not exist") + require(file.exists(), s"Properties file $file does not exist") + require(file.isFile(), s"Properties file $file is not a normal file") val inputStream = new FileInputStream(file) - val properties = new Properties() try { + val properties = new Properties() properties.load(inputStream) + properties.stringPropertyNames().toSeq.map(k => (k, properties(k).trim)) } catch { case e: IOException => - val message = s"Failed when loading Spark properties file ${file.getName}" + val message = s"Failed when loading Spark properties file $file" throw new SparkException(message, e) + } finally { + inputStream.close() } - properties.stringPropertyNames().toSeq.map(k => (k, properties(k).trim)) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 46b9f4dc7d3ba..72d0589689e71 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import java.util.Date import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import akka.actor.ActorRef @@ -36,6 +37,7 @@ private[spark] class ApplicationInfo( @transient var state: ApplicationState.Value = _ @transient var executors: mutable.HashMap[Int, ExecutorInfo] = _ + @transient var removedExecutors: ArrayBuffer[ExecutorInfo] = _ @transient var coresGranted: Int = _ @transient var endTime: Long = _ @transient var appSource: ApplicationSource = _ @@ -51,6 +53,7 @@ private[spark] class ApplicationInfo( endTime = -1L appSource = new ApplicationSource(this) nextExecutorId = 0 + removedExecutors = new ArrayBuffer[ExecutorInfo] } private def newExecutorId(useID: Option[Int] = None): Int = { @@ -74,6 +77,7 @@ private[spark] class ApplicationInfo( def removeExecutor(exec: ExecutorInfo) { if (executors.contains(exec.id)) { + removedExecutors += executors(exec.id) executors -= exec.id coresGranted -= exec.cores } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala index 76db61dd619c6..d417070c51016 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala @@ -34,4 +34,19 @@ private[spark] class ExecutorInfo( } def fullId: String = application.id + "/" + id + + override def equals(other: Any): Boolean = { + other match { + case info: ExecutorInfo => + fullId == info.fullId && + worker.id == info.worker.id && + cores == info.cores && + memory == info.memory + case _ => false + } + } + + override def toString: String = fullId + + override def hashCode: Int = toString.hashCode() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index c6dec305bffcb..33ffcbd216954 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -303,10 +303,11 @@ private[spark] class Master( appInfo.removeExecutor(exec) exec.worker.removeExecutor(exec) + val normalExit = exitStatus.exists(_ == 0) // Only retry certain number of times so we don't go into an infinite loop. - if (appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) { + if (!normalExit && appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) { schedule() - } else { + } else if (!normalExit) { logError("Application %s with ID %s failed %d times, removing it".format( appInfo.desc.name, appInfo.id, appInfo.retryCount)) removeApplication(appInfo, ApplicationState.FAILED) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index b5cd4d2ea963f..34fa1429c86de 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -25,7 +25,7 @@ import scala.xml.Node import akka.pattern.ask import org.json4s.JValue -import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.deploy.{ExecutorState, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo import org.apache.spark.ui.{WebUIPage, UIUtils} @@ -57,43 +57,55 @@ private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app }) val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") - val executors = app.executors.values.toSeq - val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) + val allExecutors = (app.executors.values ++ app.removedExecutors).toSet.toSeq + // This includes executors that are either still running or have exited cleanly + val executors = allExecutors.filter { exec => + !ExecutorState.isFinished(exec.state) || exec.state == ExecutorState.EXITED + } + val removedExecutors = allExecutors.diff(executors) + val executorsTable = UIUtils.listingTable(executorHeaders, executorRow, executors) + val removedExecutorsTable = UIUtils.listingTable(executorHeaders, executorRow, removedExecutors) val content = -
-
-
    -
  • ID: {app.id}
  • -
  • Name: {app.desc.name}
  • -
  • User: {app.desc.user}
  • -
  • Cores: - { - if (app.desc.maxCores.isEmpty) { - "Unlimited (%s granted)".format(app.coresGranted) - } else { - "%s (%s granted, %s left)".format( - app.desc.maxCores.get, app.coresGranted, app.coresLeft) - } - } -
  • -
  • - Executor Memory: - {Utils.megabytesToString(app.desc.memoryPerSlave)} -
  • -
  • Submit Date: {app.submitDate}
  • -
  • State: {app.state}
  • -
  • Application Detail UI
  • -
-
+
+
+
    +
  • ID: {app.id}
  • +
  • Name: {app.desc.name}
  • +
  • User: {app.desc.user}
  • +
  • Cores: + { + if (app.desc.maxCores.isEmpty) { + "Unlimited (%s granted)".format(app.coresGranted) + } else { + "%s (%s granted, %s left)".format( + app.desc.maxCores.get, app.coresGranted, app.coresLeft) + } + } +
  • +
  • + Executor Memory: + {Utils.megabytesToString(app.desc.memoryPerSlave)} +
  • +
  • Submit Date: {app.submitDate}
  • +
  • State: {app.state}
  • +
  • Application Detail UI
  • +
+
-
-
-

Executor Summary

- {executorTable} -
-
; +
+
+

Executor Summary

+ {executorsTable} + { + if (removedExecutors.nonEmpty) { +

Removed Executors

++ + removedExecutorsTable + } + } +
+
; UIUtils.basicSparkPage(content, "Application: " + app.desc.name) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index c7f0f244ea2fc..4af5bc3afad6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -65,6 +65,8 @@ object CommandUtils extends Logging { Seq() } + val permGenOpt = Seq("-XX:MaxPermSize=128m") + // 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( @@ -73,7 +75,7 @@ object CommandUtils extends Logging { val userClassPath = command.classPathEntries ++ Seq(classPath) Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts + permGenOpt ++ libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 2051403682737..6433aac1c23e0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -23,9 +23,10 @@ import akka.actor.ActorRef import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.util.logging.FileAppender /** * Manages the execution of one executor process. @@ -42,12 +43,15 @@ private[spark] class ExecutorRunner( val sparkHome: File, val workDir: File, val workerUrl: String, + val conf: SparkConf, var state: ExecutorState.Value) extends Logging { val fullId = appId + "/" + execId var workerThread: Thread = null var process: Process = null + var stdoutAppender: FileAppender = null + var stderrAppender: FileAppender = null // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might // make sense to remove this in the future. @@ -61,17 +65,30 @@ private[spark] class ExecutorRunner( // Shutdown hook that kills actors on shutdown. shutdownHook = new Thread() { override def run() { - killProcess() + killProcess(Some("Worker shutting down")) } } Runtime.getRuntime.addShutdownHook(shutdownHook) } - private def killProcess() { + /** + * kill executor process, wait for exit and notify worker to update resource status + * + * @param message the exception message which caused the executor's death + */ + private def killProcess(message: Option[String]) { if (process != null) { logInfo("Killing process!") process.destroy() process.waitFor() + if (stdoutAppender != null) { + stdoutAppender.stop() + } + if (stderrAppender != null) { + stderrAppender.stop() + } + val exitCode = process.waitFor() + worker ! ExecutorStateChanged(appId, execId, state, message, Some(exitCode)) } } @@ -82,7 +99,6 @@ private[spark] class ExecutorRunner( workerThread.interrupt() workerThread = null state = ExecutorState.KILLED - worker ! ExecutorStateChanged(appId, execId, state, None, None) Runtime.getRuntime.removeShutdownHook(shutdownHook) } } @@ -132,30 +148,28 @@ private[spark] class ExecutorRunner( // Redirect its stdout and stderr to files val stdout = new File(executorDir, "stdout") - CommandUtils.redirectStream(process.getInputStream, stdout) + stdoutAppender = FileAppender(process.getInputStream, stdout, conf) val stderr = new File(executorDir, "stderr") Files.write(header, stderr, Charsets.UTF_8) - CommandUtils.redirectStream(process.getErrorStream, stderr) + stderrAppender = FileAppender(process.getErrorStream, stderr, conf) - // Wait for it to exit; this is actually a bad thing if it happens, because we expect to run - // long-lived processes only. However, in the future, we might restart the executor a few - // times on the same machine. + // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) + // or with nonzero exit code val exitCode = process.waitFor() - state = ExecutorState.FAILED + state = ExecutorState.EXITED val message = "Command exited with code " + exitCode worker ! ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode)) } catch { case interrupted: InterruptedException => { logInfo("Runner thread for executor " + fullId + " interrupted") - killProcess() + state = ExecutorState.KILLED + killProcess(None) } case e: Exception => { logError("Error running executor", e) - killProcess() state = ExecutorState.FAILED - val message = e.getClass + ": " + e.getMessage - worker ! ExecutorStateChanged(appId, execId, state, Some(message), None) + killProcess(Some(e.toString)) } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index fb9cc116cd08b..a0ecaf709f8e2 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -235,7 +235,7 @@ private[spark] class Worker( val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, self, workerId, host, appDesc.sparkHome.map(userSparkHome => new File(userSparkHome)).getOrElse(sparkHome), - workDir, akkaUrl, ExecutorState.RUNNING) + workDir, akkaUrl, conf, ExecutorState.RUNNING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -263,14 +263,20 @@ private[spark] class Worker( } val fullId = appId + "/" + execId if (ExecutorState.isFinished(state)) { - val executor = executors(fullId) - logInfo("Executor " + fullId + " finished with state " + state + - message.map(" message " + _).getOrElse("") + - exitStatus.map(" exitStatus " + _).getOrElse("")) - executors -= fullId - finishedExecutors(fullId) = executor - coresUsed -= executor.cores - memoryUsed -= executor.memory + executors.get(fullId) match { + case Some(executor) => + logInfo("Executor " + fullId + " finished with state " + state + + message.map(" message " + _).getOrElse("") + + exitStatus.map(" exitStatus " + _).getOrElse("")) + executors -= fullId + finishedExecutors(fullId) = executor + coresUsed -= executor.cores + memoryUsed -= executor.memory + case None => + logInfo("Unknown Executor " + fullId + " finished with state " + state + + message.map(" message " + _).getOrElse("") + + exitStatus.map(" exitStatus " + _).getOrElse("")) + } } case KillExecutor(masterUrl, appId, execId) => @@ -311,10 +317,14 @@ private[spark] class Worker( state match { case DriverState.ERROR => logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}") + case DriverState.FAILED => + logWarning(s"Driver $driverId exited with failure") case DriverState.FINISHED => logInfo(s"Driver $driverId exited successfully") case DriverState.KILLED => logInfo(s"Driver $driverId was killed by user") + case _ => + logDebug(s"Driver $driverId changed state to $state") } masterLock.synchronized { master ! DriverStateChanged(driverId, state, exception) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 8381f59672ea3..6a5ffb1b71bfb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -24,8 +24,10 @@ import scala.xml.Node import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils +import org.apache.spark.Logging +import org.apache.spark.util.logging.{FileAppender, RollingFileAppender} -private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { +private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { private val worker = parent.worker private val workDir = parent.workDir @@ -39,21 +41,18 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { val offset = Option(request.getParameter("offset")).map(_.toLong) val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - val path = (appId, executorId, driverId) match { + val logDir = (appId, executorId, driverId) match { case (Some(a), Some(e), None) => - s"${workDir.getPath}/$appId/$executorId/$logType" + s"${workDir.getPath}/$appId/$executorId/" case (None, None, Some(d)) => - s"${workDir.getPath}/$driverId/$logType" + s"${workDir.getPath}/$driverId/" case _ => throw new Exception("Request must specify either application or driver identifiers") } - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - - val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n" - pre + Utils.offsetBytes(path, startByte, endByte) + val (logText, startByte, endByte, logLength) = getLog(logDir, logType, offset, byteLength) + val pre = s"==== Bytes $startByte-$endByte of $logLength of $logDir$logType ====\n" + pre + logText } def render(request: HttpServletRequest): Seq[Node] = { @@ -65,19 +64,16 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { val offset = Option(request.getParameter("offset")).map(_.toLong) val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - val (path, params) = (appId, executorId, driverId) match { + val (logDir, params) = (appId, executorId, driverId) match { case (Some(a), Some(e), None) => - (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e") + (s"${workDir.getPath}/$a/$e/", s"appId=$a&executorId=$e") case (None, None, Some(d)) => - (s"${workDir.getPath}/$d/$logType", s"driverId=$d") + (s"${workDir.getPath}/$d/", s"driverId=$d") case _ => throw new Exception("Request must specify either application or driver identifiers") } - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - val logText = {Utils.offsetBytes(path, startByte, endByte)} + val (logText, startByte, endByte, logLength) = getLog(logDir, logType, offset, byteLength) val linkToMaster =

Back to Master

val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} @@ -127,23 +123,37 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { UIUtils.basicSparkPage(content, logType + " log page for " + appId) } - /** Determine the byte range for a log or log page. */ - private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { - val defaultBytes = 100 * 1024 - val maxBytes = 1024 * 1024 - val file = new File(path) - val logLength = file.length() - val getOffset = offset.getOrElse(logLength - defaultBytes) - val startByte = - if (getOffset < 0) { - 0L - } else if (getOffset > logLength) { - logLength - } else { - getOffset + /** Get the part of the log files given the offset and desired length of bytes */ + private def getLog( + logDirectory: String, + logType: String, + offsetOption: Option[Long], + byteLength: Int + ): (String, Long, Long, Long) = { + try { + val files = RollingFileAppender.getSortedRolledOverFiles(logDirectory, logType) + logDebug(s"Sorted log files of type $logType in $logDirectory:\n${files.mkString("\n")}") + + val totalLength = files.map { _.length }.sum + val offset = offsetOption.getOrElse(totalLength - byteLength) + val startIndex = { + if (offset < 0) { + 0L + } else if (offset > totalLength) { + totalLength + } else { + offset + } } - val logPageLength = math.min(byteLength, maxBytes) - val endByte = math.min(startByte + logPageLength, logLength) - (startByte, endByte) + val endIndex = math.min(startIndex + totalLength, totalLength) + logDebug(s"Getting log from $startIndex to $endIndex") + val logText = Utils.offsetBytes(files, startIndex, endIndex) + logDebug(s"Got log of length ${logText.length} bytes") + (logText, startIndex, endIndex, totalLength) + } catch { + case e: Exception => + logError(s"Error getting $logType logs from directory $logDirectory", e) + ("Error getting logs due to exception: " + e.getMessage, 0, 0, 0) + } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index d4513118ced05..327b905032800 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -46,74 +46,62 @@ private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, timeout) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") + val executorHeaders = Seq("ExecutorID", "Cores", "State", "Memory", "Job Details", "Logs") + val runningExecutors = workerState.executors val runningExecutorTable = - UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) + UIUtils.listingTable(executorHeaders, executorRow, runningExecutors) + val finishedExecutors = workerState.finishedExecutors val finishedExecutorTable = - UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors) + UIUtils.listingTable(executorHeaders, executorRow, finishedExecutors) val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", "Memory", "Logs", "Notes") val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse val runningDriverTable = UIUtils.listingTable(driverHeaders, driverRow, runningDrivers) val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse - def finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers) + val finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers) // For now we only show driver information if the user has submitted drivers to the cluster. // This is until we integrate the notion of drivers and applications in the UI. - def hasDrivers = runningDrivers.length > 0 || finishedDrivers.length > 0 val content = -
-
-
    -
  • ID: {workerState.workerId}
  • -
  • - Master URL: {workerState.masterUrl} -
  • -
  • Cores: {workerState.cores} ({workerState.coresUsed} Used)
  • -
  • Memory: {Utils.megabytesToString(workerState.memory)} - ({Utils.megabytesToString(workerState.memoryUsed)} Used)
  • -
-

Back to Master

-
+
+
+
    +
  • ID: {workerState.workerId}
  • +
  • + Master URL: {workerState.masterUrl} +
  • +
  • Cores: {workerState.cores} ({workerState.coresUsed} Used)
  • +
  • Memory: {Utils.megabytesToString(workerState.memory)} + ({Utils.megabytesToString(workerState.memoryUsed)} Used)
  • +
+

Back to Master

- -
-
-

Running Executors {workerState.executors.size}

- {runningExecutorTable} -
-
- // scalastyle:off -
- {if (hasDrivers) -
-
-

Running Drivers {workerState.drivers.size}

- {runningDriverTable} -
-
+
+
+
+

Running Executors ({runningExecutors.size})

+ {runningExecutorTable} + { + if (runningDrivers.nonEmpty) { +

Running Drivers ({runningDrivers.size})

++ + runningDriverTable + } } -
- -
-
-

Finished Executors

- {finishedExecutorTable} -
-
- -
- {if (hasDrivers) -
-
-

Finished Drivers

- {finishedDriverTable} -
-
+ { + if (finishedExecutors.nonEmpty) { +

Finished Executors ({finishedExecutors.size})

++ + finishedExecutorTable + } } -
; - // scalastyle:on + { + if (finishedDrivers.nonEmpty) { +

Finished Drivers ({finishedDrivers.size})

++ + finishedDriverTable + } + } +
+
; UIUtils.basicSparkPage(content, "Spark Worker at %s:%s".format( workerState.host, workerState.port)) } @@ -122,6 +110,7 @@ private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { {executor.execId} {executor.cores} + {executor.state} {Utils.megabytesToString(executor.memory)} diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 84aec65b7765d..2279d77c91c89 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,11 +22,12 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SparkEnv, Logging, SecurityManager, SparkConf} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ +import org.apache.spark.scheduler.TaskDescription import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( @@ -61,12 +62,14 @@ private[spark] class CoarseGrainedExecutorBackend( logError("Slave registration failed: " + message) System.exit(1) - case LaunchTask(taskDesc) => - logInfo("Got assigned task " + taskDesc.taskId) + case LaunchTask(data) => if (executor == null) { logError("Received LaunchTask command but executor was null") System.exit(1) } else { + val ser = SparkEnv.get.closureSerializer.newInstance() + val taskDesc = ser.deserialize[TaskDescription](data.value) + logInfo("Got assigned task " + taskDesc.taskId) executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 3ffaaab23d0f5..3b6298a26d7c5 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -210,7 +210,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, var nextMessageToBeUsed = 0 def addMessage(message: Message) { - messages.synchronized{ + messages.synchronized { /* messages += message */ messages.enqueue(message) logDebug("Added [" + message + "] to outbox for sending to " + @@ -223,7 +223,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, while (!messages.isEmpty) { /* nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ /* val message = messages(nextMessageToBeUsed) */ - val message = messages.dequeue + val message = messages.dequeue() val chunk = message.getChunkForSending(defaultChunkSize) if (chunk.isDefined) { messages.enqueue(message) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index dcbbc1853186b..cf1c985c2fff9 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -93,7 +93,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, implicit val futureExecContext = ExecutionContext.fromExecutor( Utils.newDaemonCachedThreadPool("Connection manager future execution context")) - private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null + @volatile + private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message] = null private val authEnabled = securityManager.isAuthenticationEnabled() @@ -249,14 +250,14 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, try { while(!selectorThread.isInterrupted) { while (! registerRequests.isEmpty) { - val conn: SendingConnection = registerRequests.dequeue + val conn: SendingConnection = registerRequests.dequeue() addListeners(conn) conn.connect() addConnection(conn) } while(!keyInterestChangeRequests.isEmpty) { - val (key, ops) = keyInterestChangeRequests.dequeue + val (key, ops) = keyInterestChangeRequests.dequeue() try { if (key.isValid) { @@ -531,9 +532,9 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } return } - var securityMsgResp = SecurityMessage.fromResponse(replyToken, + val securityMsgResp = SecurityMessage.fromResponse(replyToken, securityMsg.getConnectionId.toString()) - var message = securityMsgResp.toBufferMessage + val message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security message") sendSecurityMessage(waitingConn.getRemoteConnectionManagerId(), message) } catch { @@ -567,9 +568,9 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, logDebug("Server sasl not completed: " + connection.connectionId) } if (replyToken != null) { - var securityMsgResp = SecurityMessage.fromResponse(replyToken, + val securityMsgResp = SecurityMessage.fromResponse(replyToken, securityMsg.getConnectionId) - var message = securityMsgResp.toBufferMessage + val message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security Message") sendSecurityMessage(connection.getRemoteConnectionManagerId(), message) } @@ -617,7 +618,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, return true } } - return false + false } private def handleMessage( @@ -693,9 +694,9 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, var firstResponse: Array[Byte] = null try { firstResponse = conn.sparkSaslClient.firstToken() - var securityMsg = SecurityMessage.fromResponse(firstResponse, + val securityMsg = SecurityMessage.fromResponse(firstResponse, conn.connectionId.toString()) - var message = securityMsg.toBufferMessage + val message = securityMsg.toBufferMessage if (message == null) throw new Exception("Error creating security message") connectionsAwaitingSasl += ((conn.connectionId, conn)) sendSecurityMessage(connManagerId, message) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 9ff76892aed32..5951865e56c9d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -27,6 +27,7 @@ import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleHandle private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -44,7 +45,7 @@ private[spark] case class NarrowCoGroupSplitDep( } } -private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep +private[spark] case class ShuffleCoGroupSplitDep(handle: ShuffleHandle) extends CoGroupSplitDep private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) extends Partition with Serializable { @@ -74,10 +75,11 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Seq[CoGroup] - private var serializer: Serializer = null + private var serializer: Option[Serializer] = None + /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): CoGroupedRDD[K] = { - this.serializer = serializer + this.serializer = Option(serializer) this } @@ -88,7 +90,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: new OneToOneDependency(rdd) } else { logDebug("Adding shuffle dependency with " + rdd) - new ShuffleDependency[Any, Any](rdd, part, serializer) + new ShuffleDependency[K, Any, CoGroupCombiner](rdd, part, serializer) } } } @@ -100,8 +102,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) => // Assume each RDD contributed a single dependency, and get it dependencies(j) match { - case s: ShuffleDependency[_, _] => - new ShuffleCoGroupSplitDep(s.shuffleId) + case s: ShuffleDependency[_, _, _] => + new ShuffleCoGroupSplitDep(s.shuffleHandle) case _ => new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) } @@ -126,11 +128,11 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val it = rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]] rddIterators += ((it, depNum)) - case ShuffleCoGroupSplitDep(shuffleId) => + case ShuffleCoGroupSplitDep(handle) => // Read map outputs of shuffle - val fetcher = SparkEnv.get.shuffleFetcher - val ser = Serializer.getSerializer(serializer) - val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) + val it = SparkEnv.get.shuffleManager + .getReader(handle, split.index, split.index + 1, context) + .read() rddIterators += ((it, depNum)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 6547755764dcf..2aa111d600e9b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -139,10 +139,13 @@ class HadoopRDD[K, V]( // Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the // local process. The local cache is accessed through HadoopRDD.putCachedMetadata(). // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects. - val newJobConf = new JobConf(broadcastedConf.value.value) - initLocalJobConfFuncOpt.map(f => f(newJobConf)) - HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) - newJobConf + // synchronize to prevent ConcurrentModificationException (Spark-1097, Hadoop-10456) + broadcastedConf.synchronized { + val newJobConf = new JobConf(broadcastedConf.value.value) + initLocalJobConfFuncOpt.map(f => f(newJobConf)) + HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) + newJobConf + } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 223fef79261d0..fe36c80e0be84 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -28,7 +28,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import com.clearspring.analytics.stream.cardinality.HyperLogLog +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType @@ -46,7 +46,6 @@ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer -import org.apache.spark.util.SerializableHyperLogLog /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -119,6 +118,56 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) } + /** + * Aggregate the values of each key, using given combine functions and a neutral "zero value". + * This function can return a different result type, U, than the type of the values in this RDD, + * V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, + * as in scala.TraversableOnce. The former operation is used for merging values within a + * partition, and the latter is used for merging values between partitions. To avoid memory + * allocation, both of these functions are allowed to modify and return their first argument + * instead of creating a new U. + */ + def aggregateByKey[U: ClassTag](zeroValue: U, partitioner: Partitioner)(seqOp: (U, V) => U, + combOp: (U, U) => U): RDD[(K, U)] = { + // Serialize the zero value to a byte array so that we can get a new clone of it on each key + val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroArray = new Array[Byte](zeroBuffer.limit) + zeroBuffer.get(zeroArray) + + lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + def createZero() = cachedSerializer.deserialize[U](ByteBuffer.wrap(zeroArray)) + + combineByKey[U]((v: V) => seqOp(createZero(), v), seqOp, combOp, partitioner) + } + + /** + * Aggregate the values of each key, using given combine functions and a neutral "zero value". + * This function can return a different result type, U, than the type of the values in this RDD, + * V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, + * as in scala.TraversableOnce. The former operation is used for merging values within a + * partition, and the latter is used for merging values between partitions. To avoid memory + * allocation, both of these functions are allowed to modify and return their first argument + * instead of creating a new U. + */ + def aggregateByKey[U: ClassTag](zeroValue: U, numPartitions: Int)(seqOp: (U, V) => U, + combOp: (U, U) => U): RDD[(K, U)] = { + aggregateByKey(zeroValue, new HashPartitioner(numPartitions))(seqOp, combOp) + } + + /** + * Aggregate the values of each key, using given combine functions and a neutral "zero value". + * This function can return a different result type, U, than the type of the values in this RDD, + * V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, + * as in scala.TraversableOnce. The former operation is used for merging values within a + * partition, and the latter is used for merging values between partitions. To avoid memory + * allocation, both of these functions are allowed to modify and return their first argument + * instead of creating a new U. + */ + def aggregateByKey[U: ClassTag](zeroValue: U)(seqOp: (U, V) => U, + combOp: (U, U) => U): RDD[(K, U)] = { + aggregateByKey(zeroValue, defaultPartitioner(self))(seqOp, combOp) + } + /** * Merge the values for each key using an associative function and a neutral "zero value" which * may be added to the result an arbitrary number of times, and must not change the result @@ -214,39 +263,88 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } /** + * :: Experimental :: + * * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vice versa. Uses the provided - * Partitioner to partition the output RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p` + * would trigger sparse representation of registers, which may reduce the memory consumption + * and increase accuracy when the cardinality is small. + * + * @param p The precision value for the normal set. + * `p` must be a value between 4 and `sp` if `sp` is not zero (32 max). + * @param sp The precision value for the sparse set, between 0 and 32. + * If `sp` equals 0, the sparse representation is skipped. + * @param partitioner Partitioner to use for the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { - val createHLL = (v: V) => new SerializableHyperLogLog(new HyperLogLog(relativeSD)).add(v) - val mergeValueHLL = (hll: SerializableHyperLogLog, v: V) => hll.add(v) - val mergeHLL = (h1: SerializableHyperLogLog, h2: SerializableHyperLogLog) => h1.merge(h2) + @Experimental + def countApproxDistinctByKey(p: Int, sp: Int, partitioner: Partitioner): RDD[(K, Long)] = { + require(p >= 4, s"p ($p) must be >= 4") + require(sp <= 32, s"sp ($sp) must be <= 32") + require(sp == 0 || p <= sp, s"p ($p) cannot be greater than sp ($sp)") + val createHLL = (v: V) => { + val hll = new HyperLogLogPlus(p, sp) + hll.offer(v) + hll + } + val mergeValueHLL = (hll: HyperLogLogPlus, v: V) => { + hll.offer(v) + hll + } + val mergeHLL = (h1: HyperLogLogPlus, h2: HyperLogLogPlus) => { + h1.addAll(h2) + h1 + } + + combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.cardinality()) + } - combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.value.cardinality()) + /** + * Return approximate number of distinct values for each key in this RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param partitioner partitioner of the resulting RDD + */ + def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { + require(relativeSD > 0.000017, s"accuracy ($relativeSD) must be greater than 0.000017") + val p = math.ceil(2.0 * math.log(1.054 / relativeSD) / math.log(2)).toInt + assert(p <= 32) + countApproxDistinctByKey(if (p < 4) 4 else p, 0, partitioner) } /** * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vice versa. HashPartitions the - * output RDD into numPartitions. * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param numPartitions number of partitions of the resulting RDD */ def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)] = { countApproxDistinctByKey(relativeSD, new HashPartitioner(numPartitions)) } /** - * Return approximate number of distinct values for each key this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vice versa. The default value of - * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism - * level. + * Return approximate number of distinct values for each key in this RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. */ def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = { countApproxDistinctByKey(relativeSD, defaultPartitioner(self)) @@ -689,7 +787,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val outfmt = job.getOutputFormatClass val jobFormat = outfmt.newInstance - if (jobFormat.isInstanceOf[NewFileOutputFormat[_, _]]) { + if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) { // FileOutputFormat ignores the filesystem parameter jobFormat.checkOutputSpecs(job) } @@ -755,7 +853,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + valueClass.getSimpleName + ")") - if (outputFormatInstance.isInstanceOf[FileOutputFormat[_, _]]) { + if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) { // FileOutputFormat ignores the filesystem parameter val ignoredFs = FileSystem.get(conf) conf.getOutputFormat.checkOutputSpecs(ignoredFs, conf) diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 2425929fc73c5..66c71bf7e8bb5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -117,6 +117,15 @@ private object ParallelCollectionRDD { if (numSlices < 1) { throw new IllegalArgumentException("Positive number of slices required") } + // Sequences need to be sliced at the same set of index positions for operations + // like RDD.zip() to behave as expected + def positions(length: Long, numSlices: Int): Iterator[(Int, Int)] = { + (0 until numSlices).iterator.map(i => { + val start = ((i * length) / numSlices).toInt + val end = (((i + 1) * length) / numSlices).toInt + (start, end) + }) + } seq match { case r: Range.Inclusive => { val sign = if (r.step < 0) { @@ -128,18 +137,17 @@ private object ParallelCollectionRDD { r.start, r.end + sign, r.step).asInstanceOf[Seq[T]], numSlices) } case r: Range => { - (0 until numSlices).map(i => { - val start = ((i * r.length.toLong) / numSlices).toInt - val end = (((i + 1) * r.length.toLong) / numSlices).toInt - new Range(r.start + start * r.step, r.start + end * r.step, r.step) - }).asInstanceOf[Seq[Seq[T]]] + positions(r.length, numSlices).map({ + case (start, end) => + new Range(r.start + start * r.step, r.start + end * r.step, r.step) + }).toSeq.asInstanceOf[Seq[Seq[T]]] } case nr: NumericRange[_] => { // For ranges of Long, Double, BigInteger, etc val slices = new ArrayBuffer[Seq[T]](numSlices) - val sliceSize = (nr.size + numSlices - 1) / numSlices // Round up to catch everything var r = nr - for (i <- 0 until numSlices) { + for ((start, end) <- positions(nr.length, numSlices)) { + val sliceSize = end - start slices += r.take(sliceSize).asInstanceOf[Seq[T]] r = r.drop(sliceSize) } @@ -147,11 +155,10 @@ private object ParallelCollectionRDD { } case _ => { val array = seq.toArray // To prevent O(n^2) operations for List etc - (0 until numSlices).map(i => { - val start = ((i * array.length.toLong) / numSlices).toInt - val end = (((i + 1) * array.length.toLong) / numSlices).toInt - array.slice(start, end).toSeq - }) + positions(array.length, numSlices).map({ + case (start, end) => + array.slice(start, end).toSeq + }).toSeq } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index aa03e9276fb34..1633b185861b9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -19,12 +19,11 @@ package org.apache.spark.rdd import java.util.Random -import scala.collection.Map -import scala.collection.mutable +import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} -import com.clearspring.analytics.stream.cardinality.HyperLogLog +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable @@ -41,9 +40,9 @@ import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, SerializableHyperLogLog, Utils} +import org.apache.spark.util.{BoundedPriorityQueue, CallSite, Utils} import org.apache.spark.util.collection.OpenHashMap -import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler} +import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} /** * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, @@ -379,46 +378,56 @@ abstract class RDD[T: ClassTag]( }.toArray } - def takeSample(withReplacement: Boolean, num: Int, seed: Long = Utils.random.nextLong): Array[T] = - { - var fraction = 0.0 - var total = 0 - val multiplier = 3.0 - val initialCount = this.count() - var maxSelected = 0 + /** + * Return a fixed-size sampled subset of this RDD in an array + * + * @param withReplacement whether sampling is done with replacement + * @param num size of the returned sample + * @param seed seed for the random number generator + * @return sample of specified size in an array + */ + def takeSample(withReplacement: Boolean, + num: Int, + seed: Long = Utils.random.nextLong): Array[T] = { + val numStDev = 10.0 if (num < 0) { throw new IllegalArgumentException("Negative number of elements requested") + } else if (num == 0) { + return new Array[T](0) } + val initialCount = this.count() if (initialCount == 0) { return new Array[T](0) } - if (initialCount > Integer.MAX_VALUE - 1) { - maxSelected = Integer.MAX_VALUE - 1 - } else { - maxSelected = initialCount.toInt + val maxSampleSize = Int.MaxValue - (numStDev * math.sqrt(Int.MaxValue)).toInt + if (num > maxSampleSize) { + throw new IllegalArgumentException("Cannot support a sample size > Int.MaxValue - " + + s"$numStDev * math.sqrt(Int.MaxValue)") } - if (num > initialCount && !withReplacement) { - total = maxSelected - fraction = multiplier * (maxSelected + 1) / initialCount - } else { - fraction = multiplier * (num + 1) / initialCount - total = num + val rand = new Random(seed) + if (!withReplacement && num >= initialCount) { + return Utils.randomizeInPlace(this.collect(), rand) } - val rand = new Random(seed) + val fraction = SamplingUtils.computeFractionForSampleSize(num, initialCount, + withReplacement) + var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() // If the first sample didn't turn out large enough, keep trying to take samples; // this shouldn't happen often because we use a big multiplier for the initial size - while (samples.length < total) { + var numIters = 0 + while (samples.length < num) { + logWarning(s"Needed to re-sample due to insufficient sample size. Repeat #$numIters") samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() + numIters += 1 } - Utils.randomizeInPlace(samples, rand).take(total) + Utils.randomizeInPlace(samples, rand).take(num) } /** @@ -433,6 +442,18 @@ abstract class RDD[T: ClassTag]( */ def ++(other: RDD[T]): RDD[T] = this.union(other) + /** + * Return this RDD sorted by the given key function. + */ + def sortBy[K]( + f: (T) ⇒ K, + ascending: Boolean = true, + numPartitions: Int = this.partitions.size) + (implicit ord: Ordering[K], ctag: ClassTag[K]): RDD[T] = + this.keyBy[K](f) + .sortByKey(ascending, numPartitions) + .values + /** * Return the intersection of this RDD and another one. The output will not contain any duplicate * elements, even if the input RDDs did. @@ -655,7 +676,19 @@ abstract class RDD[T: ClassTag]( * partitions* and the *same number of elements in each partition* (e.g. one was made through * a map on the other). */ - def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) + def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = { + zipPartitions(other, true) { (thisIter, otherIter) => + new Iterator[(T, U)] { + def hasNext = (thisIter.hasNext, otherIter.hasNext) match { + case (true, true) => true + case (false, false) => false + case _ => throw new SparkException("Can only zip RDDs with " + + "same number of elements in each partition") + } + def next = (thisIter.next, otherIter.next) + } + } + } /** * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by @@ -921,15 +954,49 @@ abstract class RDD[T: ClassTag]( * :: Experimental :: * Return approximate number of distinct elements in the RDD. * - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. The default value of - * relativeSD is 0.05. + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p` + * would trigger sparse representation of registers, which may reduce the memory consumption + * and increase accuracy when the cardinality is small. + * + * @param p The precision value for the normal set. + * `p` must be a value between 4 and `sp` if `sp` is not zero (32 max). + * @param sp The precision value for the sparse set, between 0 and 32. + * If `sp` equals 0, the sparse representation is skipped. */ @Experimental + def countApproxDistinct(p: Int, sp: Int): Long = { + require(p >= 4, s"p ($p) must be greater than 0") + require(sp <= 32, s"sp ($sp) cannot be greater than 32") + require(sp == 0 || p <= sp, s"p ($p) cannot be greater than sp ($sp)") + val zeroCounter = new HyperLogLogPlus(p, sp) + aggregate(zeroCounter)( + (hll: HyperLogLogPlus, v: T) => { + hll.offer(v) + hll + }, + (h1: HyperLogLogPlus, h2: HyperLogLogPlus) => { + h1.addAll(h2) + h2 + }).cardinality() + } + + /** + * Return approximate number of distinct elements in the RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + */ def countApproxDistinct(relativeSD: Double = 0.05): Long = { - val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) - aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() + val p = math.ceil(2.0 * math.log(1.054 / relativeSD) / math.log(2)).toInt + countApproxDistinct(p, 0) } /** @@ -1007,11 +1074,11 @@ abstract class RDD[T: ClassTag]( * Returns the top K (largest) elements from this RDD as defined by the specified * implicit Ordering[T]. This does the opposite of [[takeOrdered]]. For example: * {{{ - * sc.parallelize([10, 4, 2, 12, 3]).top(1) - * // returns [12] + * sc.parallelize(Seq(10, 4, 2, 12, 3)).top(1) + * // returns Array(12) * - * sc.parallelize([2, 3, 4, 5, 6]).top(2) - * // returns [6, 5] + * sc.parallelize(Seq(2, 3, 4, 5, 6)).top(2) + * // returns Array(6, 5) * }}} * * @param num the number of top elements to return @@ -1025,11 +1092,11 @@ abstract class RDD[T: ClassTag]( * implicit Ordering[T] and maintains the ordering. This does the opposite of [[top]]. * For example: * {{{ - * sc.parallelize([10, 4, 2, 12, 3]).takeOrdered(1) - * // returns [12] + * sc.parallelize(Seq(10, 4, 2, 12, 3)).takeOrdered(1) + * // returns Array(2) * - * sc.parallelize([2, 3, 4, 5, 6]).takeOrdered(2) - * // returns [2, 3] + * sc.parallelize(Seq(2, 3, 4, 5, 6)).takeOrdered(2) + * // returns Array(2, 3) * }}} * * @param num the number of top elements to return @@ -1134,8 +1201,8 @@ abstract class RDD[T: ClassTag]( private var storageLevel: StorageLevel = StorageLevel.NONE /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ - @transient private[spark] val creationSiteInfo = Utils.getCallSiteInfo - private[spark] def getCreationSite: String = creationSiteInfo.toString + @transient private[spark] val creationSite = Utils.getCallSite + private[spark] def getCreationSite: String = Option(creationSite).map(_.short).getOrElse("") private[spark] def elementClassTag: ClassTag[T] = classTag[T] diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 802b0bdfb2d59..bb108ef163c56 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -42,10 +42,11 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( part: Partitioner) extends RDD[P](prev.context, Nil) { - private var serializer: Serializer = null + private var serializer: Option[Serializer] = None + /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): ShuffledRDD[K, V, P] = { - this.serializer = serializer + this.serializer = Option(serializer) this } @@ -60,9 +61,10 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( } override def compute(split: Partition, context: TaskContext): Iterator[P] = { - val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - val ser = Serializer.getSerializer(serializer) - SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context, ser) + val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, V]] + SparkEnv.get.shuffleManager.getReader(dep.shuffleHandle, split.index, split.index + 1, context) + .read() + .asInstanceOf[Iterator[P]] } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index 9a09c05bbc959..ed24ea22a661c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -54,10 +54,11 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { - private var serializer: Serializer = null + private var serializer: Option[Serializer] = None + /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): SubtractedRDD[K, V, W] = { - this.serializer = serializer + this.serializer = Option(serializer) this } @@ -79,8 +80,8 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( // Each CoGroupPartition will depend on rdd1 and rdd2 array(i) = new CoGroupPartition(i, Seq(rdd1, rdd2).zipWithIndex.map { case (rdd, j) => dependencies(j) match { - case s: ShuffleDependency[_, _] => - new ShuffleCoGroupSplitDep(s.shuffleId) + case s: ShuffleDependency[_, _, _] => + new ShuffleCoGroupSplitDep(s.shuffleHandle) case _ => new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) } @@ -93,7 +94,6 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] - val ser = Serializer.getSerializer(serializer) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) @@ -109,9 +109,10 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( case NarrowCoGroupSplitDep(rdd, _, itsSplit) => rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op) - case ShuffleCoGroupSplitDep(shuffleId) => - val iter = SparkEnv.get.shuffleFetcher.fetch[Product2[K, V]](shuffleId, partition.index, - context, ser) + case ShuffleCoGroupSplitDep(handle) => + val iter = SparkEnv.get.shuffleManager + .getReader(handle, partition.index, partition.index + 1, context) + .read() iter.foreach(op) } // the first dep is rdd1; add all values to the map diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala deleted file mode 100644 index b8110ffc42f2d..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import java.io.{IOException, ObjectOutputStream} - -import scala.reflect.ClassTag - -import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} - -private[spark] class ZippedPartition[T: ClassTag, U: ClassTag]( - idx: Int, - @transient rdd1: RDD[T], - @transient rdd2: RDD[U] - ) extends Partition { - - var partition1 = rdd1.partitions(idx) - var partition2 = rdd2.partitions(idx) - override val index: Int = idx - - def partitions = (partition1, partition2) - - @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { - // Update the reference to parent partition at the time of task serialization - partition1 = rdd1.partitions(idx) - partition2 = rdd2.partitions(idx) - oos.defaultWriteObject() - } -} - -private[spark] class ZippedRDD[T: ClassTag, U: ClassTag]( - sc: SparkContext, - var rdd1: RDD[T], - var rdd2: RDD[U]) - extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) { - - override def getPartitions: Array[Partition] = { - if (rdd1.partitions.size != rdd2.partitions.size) { - throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") - } - val array = new Array[Partition](rdd1.partitions.size) - for (i <- 0 until rdd1.partitions.size) { - array(i) = new ZippedPartition(i, rdd1, rdd2) - } - array - } - - override def compute(s: Partition, context: TaskContext): Iterator[(T, U)] = { - val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions - rdd1.iterator(partition1, context).zip(rdd2.iterator(partition2, context)) - } - - override def getPreferredLocations(s: Partition): Seq[String] = { - val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions - val pref1 = rdd1.preferredLocations(partition1) - val pref2 = rdd2.preferredLocations(partition2) - // Check whether there are any hosts that match both RDDs; otherwise return the union - val exactMatchLocations = pref1.intersect(pref2) - if (!exactMatchLocations.isEmpty) { - exactMatchLocations - } else { - (pref1 ++ pref2).distinct - } - } - - override def clearDependencies() { - super.clearDependencies() - rdd1 = null - rdd2 = null - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala index 9257f48559c9e..b755d8fb15757 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.util.Properties import org.apache.spark.TaskContext +import org.apache.spark.util.CallSite /** * Tracks information about an active job in the DAGScheduler. @@ -29,7 +30,7 @@ private[spark] class ActiveJob( val finalStage: Stage, val func: (TaskContext, Iterator[_]) => _, val partitions: Array[Int], - val callSite: String, + val callSite: CallSite, val listener: JobListener, val properties: Properties) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ff411e24a3d85..b3ebaa547de0d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.io.NotSerializableException +import java.io.{NotSerializableException, PrintWriter, StringWriter} import java.util.Properties import java.util.concurrent.atomic.AtomicInteger @@ -38,7 +38,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} -import org.apache.spark.util.Utils +import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils} /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -61,7 +61,8 @@ class DAGScheduler( listenerBus: LiveListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, - env: SparkEnv) + env: SparkEnv, + clock: Clock = SystemClock) extends Logging { import DAGScheduler._ @@ -189,12 +190,14 @@ class DAGScheduler( * The jobId value passed in will be used if the stage doesn't already exist with * a lower jobId (jobId always increases across jobs.) */ - private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], jobId: Int): Stage = { + private def getShuffleMapStage(shuffleDep: ShuffleDependency[_, _, _], jobId: Int): Stage = { shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => val stage = - newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId) + newOrUsedStage( + shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId, + shuffleDep.rdd.creationSite) shuffleToMapStage(shuffleDep.shuffleId) = stage stage } @@ -209,9 +212,9 @@ class DAGScheduler( private def newStage( rdd: RDD[_], numTasks: Int, - shuffleDep: Option[ShuffleDependency[_,_]], + shuffleDep: Option[ShuffleDependency[_, _, _]], jobId: Int, - callSite: Option[String] = None) + callSite: CallSite) : Stage = { val id = nextStageId.getAndIncrement() @@ -232,9 +235,9 @@ class DAGScheduler( private def newOrUsedStage( rdd: RDD[_], numTasks: Int, - shuffleDep: ShuffleDependency[_,_], + shuffleDep: ShuffleDependency[_, _, _], jobId: Int, - callSite: Option[String] = None) + callSite: CallSite) : Stage = { val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) @@ -268,7 +271,7 @@ class DAGScheduler( // we can't do it in its constructor because # of partitions is unknown for (dep <- r.dependencies) { dep match { - case shufDep: ShuffleDependency[_,_] => + case shufDep: ShuffleDependency[_, _, _] => parents += getShuffleMapStage(shufDep, jobId) case _ => visit(dep.rdd) @@ -289,7 +292,7 @@ class DAGScheduler( if (getCacheLocs(rdd).contains(Nil)) { for (dep <- rdd.dependencies) { dep match { - case shufDep: ShuffleDependency[_,_] => + case shufDep: ShuffleDependency[_, _, _] => val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { missing += mapStage @@ -412,7 +415,7 @@ class DAGScheduler( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], - callSite: String, + callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, properties: Properties = null): JobWaiter[U] = @@ -442,7 +445,7 @@ class DAGScheduler( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], - callSite: String, + callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, properties: Properties = null) @@ -451,7 +454,7 @@ class DAGScheduler( waiter.awaitResult() match { case JobSucceeded => {} case JobFailed(exception: Exception) => - logInfo("Failed to run " + callSite) + logInfo("Failed to run " + callSite.short) throw exception } } @@ -460,7 +463,7 @@ class DAGScheduler( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], - callSite: String, + callSite: CallSite, timeout: Long, properties: Properties = null) : PartialResult[R] = @@ -580,6 +583,10 @@ class DAGScheduler( case e: Exception => jobResult = JobFailed(e) job.listener.jobFailed(e) + case oom: OutOfMemoryError => + val exception = new SparkException("Local job aborted due to out of memory error", oom) + jobResult = JobFailed(exception) + job.listener.jobFailed(exception) } finally { val s = job.finalStage stageIdToJobIds -= s.id // clean up data structures that were populated for a local job, @@ -661,7 +668,7 @@ class DAGScheduler( func: (TaskContext, Iterator[_]) => _, partitions: Array[Int], allowLocal: Boolean, - callSite: String, + callSite: CallSite, listener: JobListener, properties: Properties = null) { @@ -669,7 +676,7 @@ class DAGScheduler( try { // New stage creation may throw an exception if, for example, jobs are run on a // HadoopRDD whose underlying HDFS files have been deleted. - finalStage = newStage(finalRDD, partitions.size, None, jobId, Some(callSite)) + finalStage = newStage(finalRDD, partitions.size, None, jobId, callSite) } catch { case e: Exception => logWarning("Creating new stage failed due to exception - job: " + jobId, e) @@ -680,7 +687,7 @@ class DAGScheduler( val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() logInfo("Got job %s (%s) with %d output partitions (allowLocal=%s)".format( - job.jobId, callSite, partitions.length, allowLocal)) + job.jobId, callSite.short, partitions.length, allowLocal)) logInfo("Final stage: " + finalStage + "(" + finalStage.name + ")") logInfo("Parents of final stage: " + finalStage.parents) logInfo("Missing parents: " + getMissingParentStages(finalStage)) @@ -777,7 +784,7 @@ class DAGScheduler( logDebug("New pending tasks: " + myPending) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) - stageToInfos(stage).submissionTime = Some(System.currentTimeMillis()) + stageToInfos(stage).submissionTime = Some(clock.getTime()) } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) @@ -803,11 +810,11 @@ class DAGScheduler( def markStageAsFinished(stage: Stage) = { val serviceTime = stageToInfos(stage).submissionTime match { - case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0) + case Some(t) => "%.03f".format((clock.getTime() - t) / 1000.0) case _ => "Unknown" } logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stageToInfos(stage).completionTime = Some(System.currentTimeMillis()) + stageToInfos(stage).completionTime = Some(clock.getTime()) listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) runningStages -= stage } @@ -1011,7 +1018,7 @@ class DAGScheduler( return } val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq - stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) + stageToInfos(failedStage).completionTime = Some(clock.getTime()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason", @@ -1083,7 +1090,7 @@ class DAGScheduler( visitedRdds += rdd for (dep <- rdd.dependencies) { dep match { - case shufDep: ShuffleDependency[_,_] => + case shufDep: ShuffleDependency[_, _, _] => val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { visitedStages += mapStage diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 23f57441b4b11..2b6f7e4205c32 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -25,6 +25,7 @@ import scala.language.existentials import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD +import org.apache.spark.util.CallSite /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue @@ -40,7 +41,7 @@ private[scheduler] case class JobSubmitted( func: (TaskContext, Iterator[_]) => _, partitions: Array[Int], allowLocal: Boolean, - callSite: String, + callSite: CallSite, listener: JobListener, properties: Properties = null) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index ed0f56f1abdf5..0098b5a59d1a5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -29,6 +29,7 @@ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.serializer.Serializer import org.apache.spark.storage._ +import org.apache.spark.shuffle.ShuffleWriter private[spark] object ShuffleMapTask { @@ -37,7 +38,7 @@ private[spark] object ShuffleMapTask { // expensive on the master node if it needs to launch thousands of tasks. private val serializedInfoCache = new HashMap[Int, Array[Byte]] - def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = { + def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_, _, _]): Array[Byte] = { synchronized { val old = serializedInfoCache.get(stageId).orNull if (old != null) { @@ -56,12 +57,12 @@ private[spark] object ShuffleMapTask { } } - def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_,_]) = { + def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_, _, _]) = { val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) val ser = SparkEnv.get.closureSerializer.newInstance() val objIn = ser.deserializeStream(in) val rdd = objIn.readObject().asInstanceOf[RDD[_]] - val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_]] + val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_, _, _]] (rdd, dep) } @@ -99,7 +100,7 @@ private[spark] object ShuffleMapTask { private[spark] class ShuffleMapTask( stageId: Int, var rdd: RDD[_], - var dep: ShuffleDependency[_,_], + var dep: ShuffleDependency[_, _, _], _partitionId: Int, @transient private var locs: Seq[TaskLocation]) extends Task[MapStatus](stageId, _partitionId) @@ -141,66 +142,22 @@ private[spark] class ShuffleMapTask( } override def runTask(context: TaskContext): MapStatus = { - val numOutputSplits = dep.partitioner.numPartitions metrics = Some(context.taskMetrics) - - val blockManager = SparkEnv.get.blockManager - val shuffleBlockManager = blockManager.shuffleBlockManager - var shuffle: ShuffleWriterGroup = null - var success = false - + var writer: ShuffleWriter[Any, Any] = null try { - // Obtain all the block writers for shuffle blocks. - val ser = Serializer.getSerializer(dep.serializer) - shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser) - - // Write the map output to its associated buckets. + val manager = SparkEnv.get.shuffleManager + writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) for (elem <- rdd.iterator(split, context)) { - val pair = elem.asInstanceOf[Product2[Any, Any]] - val bucketId = dep.partitioner.getPartition(pair._1) - shuffle.writers(bucketId).write(pair) - } - - // Commit the writes. Get the size of each bucket block (total block size). - var totalBytes = 0L - var totalTime = 0L - val compressedSizes: Array[Byte] = shuffle.writers.map { writer: BlockObjectWriter => - writer.commit() - writer.close() - val size = writer.fileSegment().length - totalBytes += size - totalTime += writer.timeWriting() - MapOutputTracker.compressSize(size) + writer.write(elem.asInstanceOf[Product2[Any, Any]]) } - - // Update shuffle metrics. - val shuffleMetrics = new ShuffleWriteMetrics - shuffleMetrics.shuffleBytesWritten = totalBytes - shuffleMetrics.shuffleWriteTime = totalTime - metrics.get.shuffleWriteMetrics = Some(shuffleMetrics) - - success = true - new MapStatus(blockManager.blockManagerId, compressedSizes) - } catch { case e: Exception => - // If there is an exception from running the task, revert the partial writes - // and throw the exception upstream to Spark. - if (shuffle != null && shuffle.writers != null) { - for (writer <- shuffle.writers) { - writer.revertPartialWrites() - writer.close() + return writer.stop(success = true).get + } catch { + case e: Exception => + if (writer != null) { + writer.stop(success = false) } - } - throw e + throw e } finally { - // Release the writers back to the shuffle block manager. - if (shuffle != null && shuffle.writers != null) { - try { - shuffle.releaseWriters(success) - } catch { - case e: Exception => logError("Failed to release shuffle writers", e) - } - } - // Execute the callbacks on task completion. context.executeOnCompleteCallbacks() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 5c1fc30e4a557..9a4be43ee219f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.CallSite /** * A stage is a set of independent tasks all computing the same function that need to run as part @@ -35,15 +36,20 @@ import org.apache.spark.storage.BlockManagerId * Each Stage also has a jobId, identifying the job that first submitted the stage. When FIFO * scheduling is used, this allows Stages from earlier jobs to be computed first or recovered * faster on failure. + * + * The callSite provides a location in user code which relates to the stage. For a shuffle map + * stage, the callSite gives the user code that created the RDD being shuffled. For a result + * stage, the callSite gives the user code that executes the associated action (e.g. count()). + * */ private[spark] class Stage( val id: Int, val rdd: RDD[_], val numTasks: Int, - val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage + val shuffleDep: Option[ShuffleDependency[_, _, _]], // Output shuffle if stage is a map stage val parents: List[Stage], val jobId: Int, - callSite: Option[String]) + val callSite: CallSite) extends Logging { val isShuffleMap = shuffleDep.isDefined @@ -100,7 +106,8 @@ private[spark] class Stage( id } - val name = callSite.getOrElse(rdd.getCreationSite) + val name = callSite.short + val details = callSite.long override def toString = "Stage " + id diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index b42e231e11f91..7644e3f351b3c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -25,7 +25,12 @@ import org.apache.spark.storage.RDDInfo * Stores information about a stage to pass from the scheduler to SparkListeners. */ @DeveloperApi -class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfos: Seq[RDDInfo]) { +class StageInfo( + val stageId: Int, + val name: String, + val numTasks: Int, + val rddInfos: Seq[RDDInfo], + val details: String) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None /** Time when all tasks in the stage completed or when the stage was cancelled. */ @@ -52,6 +57,6 @@ private[spark] object StageInfo { def fromStage(stage: Stage): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos - new StageInfo(stage.id, stage.name, stage.numTasks, rddInfos) + new StageInfo(stage.id, stage.name, stage.numTasks, rddInfos, stage.details) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 99d305b36a959..df59f444b7a0e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -71,7 +71,8 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) case ex: Exception => - taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) + logError("Exception while getting task result", ex) + taskSetManager.abort("Exception while getting task result: %s".format(ex)) } } }) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index ddbc74e82ac49..ca74069ef885c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -28,7 +28,7 @@ private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable private[spark] object CoarseGrainedClusterMessages { // Driver to executors - case class LaunchTask(task: TaskDescription) extends CoarseGrainedClusterMessage + case class LaunchTask(data: SerializableBuffer) extends CoarseGrainedClusterMessage case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index a6d6b3d26a3c6..e47a060683a2d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,10 +27,10 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{Logging, SparkException, TaskState} +import org.apache.spark.{SparkEnv, Logging, SparkException, TaskState} import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{SerializableBuffer, AkkaUtils, Utils} /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -48,6 +48,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A var totalCoreCount = new AtomicInteger(0) val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) + private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] @@ -140,8 +141,26 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A // Launch tasks returned by a set of resource offers def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { - freeCores(task.executorId) -= scheduler.CPUS_PER_TASK - executorActor(task.executorId) ! LaunchTask(task) + val ser = SparkEnv.get.closureSerializer.newInstance() + val serializedTask = ser.serialize(task) + if (serializedTask.limit >= akkaFrameSize - 1024) { + val taskSetId = scheduler.taskIdToTaskSetId(task.taskId) + scheduler.activeTaskSets.get(taskSetId).foreach { taskSet => + try { + var msg = "Serialized task %s:%d was %d bytes which " + + "exceeds spark.akka.frameSize (%d bytes). " + + "Consider using broadcast variables for large values." + msg = msg.format(task.taskId, task.index, serializedTask.limit, akkaFrameSize) + taskSet.abort(msg) + } catch { + case e: Exception => logError("Exception in error callback", e) + } + } + } + else { + freeCores(task.executorId) -= scheduler.CPUS_PER_TASK + executorActor(task.executorId) ! LaunchTask(new SerializableBuffer(serializedTask)) + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 933f6e0571518..9c07b3f7b695a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -46,12 +46,13 @@ private[spark] class SparkDeploySchedulerBackend( CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") - val classPathEntries = sys.props.get("spark.executor.extraClassPath").toSeq.flatMap { cp => - cp.split(java.io.File.pathSeparator) - } - val libraryPathEntries = sys.props.get("spark.executor.extraLibraryPath").toSeq.flatMap { cp => + val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath").toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } + val libraryPathEntries = + sc.conf.getOption("spark.executor.extraLibraryPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, @@ -87,6 +88,8 @@ private[spark] class SparkDeploySchedulerBackend( if (!stopping) { logError("Application has been killed. Reason: " + reason) scheduler.error(reason) + // Ensure the application terminates, as we can no longer run jobs. + sc.stop() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index cbe9bb093d1c9..9f45400bcf852 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -207,10 +207,12 @@ private[spark] class CoarseMesosSchedulerBackend( .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", sc.executorMemory)) .build() - d.launchTasks(offer.getId, Collections.singletonList(task), filters) + d.launchTasks( + Collections.singleton(offer.getId), Collections.singletonList(task), filters) } else { // Filter it out - d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters) + d.launchTasks( + Collections.singleton(offer.getId), Collections.emptyList[MesosTaskInfo](), filters) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index f08b19e6782e3..a089a02d42170 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -223,7 +223,7 @@ private[spark] class MesosSchedulerBackend( // 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) + d.launchTasks(Collections.singleton(offers(i).getId), mesosTasks(i), filters) } } } finally { diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index ee26970a3d874..f2f5cea469c61 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -52,6 +52,10 @@ object Serializer { def getSerializer(serializer: Serializer): Serializer = { if (serializer == null) SparkEnv.get.serializer else serializer } + + def getSerializer(serializer: Option[Serializer]): Serializer = { + serializer.getOrElse(SparkEnv.get.serializer) + } } diff --git a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala similarity index 66% rename from core/src/main/scala/org/apache/spark/ShuffleFetcher.scala rename to core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala index a4f69b6b22b2c..b36c457d6d514 100644 --- a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala @@ -15,22 +15,16 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.shuffle +import org.apache.spark.{ShuffleDependency, Aggregator, Partitioner} import org.apache.spark.serializer.Serializer -private[spark] abstract class ShuffleFetcher { - - /** - * Fetch the shuffle outputs for a given ShuffleDependency. - * @return An iterator over the elements of the fetched shuffle outputs. - */ - def fetch[T]( - shuffleId: Int, - reduceId: Int, - context: TaskContext, - serializer: Serializer = SparkEnv.get.serializer): Iterator[T] - - /** Stop the fetcher */ - def stop() {} -} +/** + * A basic ShuffleHandle implementation that just captures registerShuffle's parameters. + */ +private[spark] class BaseShuffleHandle[K, V, C]( + shuffleId: Int, + val numMaps: Int, + val dependency: ShuffleDependency[K, V, C]) + extends ShuffleHandle(shuffleId) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala new file mode 100644 index 0000000000000..13c7115f88afa --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +/** + * An opaque handle to a shuffle, used by a ShuffleManager to pass information about it to tasks. + * + * @param shuffleId ID of the shuffle + */ +private[spark] abstract class ShuffleHandle(val shuffleId: Int) extends Serializable {} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala new file mode 100644 index 0000000000000..9c859b8b4a118 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import org.apache.spark.{TaskContext, ShuffleDependency} + +/** + * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on both the + * driver and executors, based on the spark.shuffle.manager setting. The driver registers shuffles + * with it, and executors (or tasks running locally in the driver) can ask to read and write data. + * + * NOTE: this will be instantiated by SparkEnv so its constructor can take a SparkConf and + * boolean isDriver as parameters. + */ +private[spark] trait ShuffleManager { + /** + * Register a shuffle with the manager and obtain a handle for it to pass to tasks. + */ + def registerShuffle[K, V, C]( + shuffleId: Int, + numMaps: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle + + /** Get a writer for a given partition. Called on executors by map tasks. */ + def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext): ShuffleWriter[K, V] + + /** + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). + * Called on executors by reduce tasks. + */ + def getReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext): ShuffleReader[K, C] + + /** Remove a shuffle's metadata from the ShuffleManager. */ + def unregisterShuffle(shuffleId: Int) + + /** Shut down this ShuffleManager. */ + def stop(): Unit +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala new file mode 100644 index 0000000000000..b30e366d06006 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +/** + * Obtained inside a reduce task to read combined records from the mappers. + */ +private[spark] trait ShuffleReader[K, C] { + /** Read the combined key-values for this reduce task */ + def read(): Iterator[Product2[K, C]] + + /** Close this reader */ + def stop(): Unit +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala new file mode 100644 index 0000000000000..ead3ebd652ca5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import org.apache.spark.scheduler.MapStatus + +/** + * Obtained inside a map task to write out records to the shuffle system. + */ +private[spark] trait ShuffleWriter[K, V] { + /** Write a record to this task's output */ + def write(record: Product2[K, V]): Unit + + /** Close this writer, passing along whether the map completed */ + def stop(success: Boolean): Option[MapStatus] +} diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala rename to core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index a67392441ed29..b05b6ea345df3 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.shuffle.hash import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -24,17 +24,16 @@ import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} import org.apache.spark.util.CompletionIterator +import org.apache.spark._ -private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { - - override def fetch[T]( +private[hash] object BlockStoreShuffleFetcher extends Logging { + def fetch[T]( shuffleId: Int, reduceId: Int, context: TaskContext, serializer: Serializer) : Iterator[T] = { - logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala new file mode 100644 index 0000000000000..5b0940ecce29d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.hash + +import org.apache.spark._ +import org.apache.spark.shuffle._ + +/** + * A ShuffleManager using hashing, that creates one output file per reduce partition on each + * mapper (possibly reusing these across waves of tasks). + */ +class HashShuffleManager(conf: SparkConf) extends ShuffleManager { + /* Register a shuffle with the manager and obtain a handle for it to pass to tasks. */ + override def registerShuffle[K, V, C]( + shuffleId: Int, + numMaps: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + new BaseShuffleHandle(shuffleId, numMaps, dependency) + } + + /** + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). + * Called on executors by reduce tasks. + */ + override def getReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext): ShuffleReader[K, C] = { + new HashShuffleReader( + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, endPartition, context) + } + + /** Get a writer for a given partition. Called on executors by map tasks. */ + override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext) + : ShuffleWriter[K, V] = { + new HashShuffleWriter(handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context) + } + + /** Remove a shuffle's metadata from the ShuffleManager. */ + override def unregisterShuffle(shuffleId: Int): Unit = {} + + /** Shut down this ShuffleManager. */ + override def stop(): Unit = {} +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala new file mode 100644 index 0000000000000..f6a790309a587 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.hash + +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} +import org.apache.spark.TaskContext + +class HashShuffleReader[K, C]( + handle: BaseShuffleHandle[K, _, C], + startPartition: Int, + endPartition: Int, + context: TaskContext) + extends ShuffleReader[K, C] +{ + require(endPartition == startPartition + 1, + "Hash shuffle currently only supports fetching one partition") + + /** Read the combined key-values for this reduce task */ + override def read(): Iterator[Product2[K, C]] = { + BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, + Serializer.getSerializer(handle.dependency.serializer)) + } + + /** Close this reader */ + override def stop(): Unit = ??? +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala new file mode 100644 index 0000000000000..4c6749098c110 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.hash + +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleWriter} +import org.apache.spark.{Logging, MapOutputTracker, SparkEnv, TaskContext} +import org.apache.spark.storage.{BlockObjectWriter} +import org.apache.spark.serializer.Serializer +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.scheduler.MapStatus + +class HashShuffleWriter[K, V]( + handle: BaseShuffleHandle[K, V, _], + mapId: Int, + context: TaskContext) + extends ShuffleWriter[K, V] with Logging { + + private val dep = handle.dependency + private val numOutputSplits = dep.partitioner.numPartitions + private val metrics = context.taskMetrics + private var stopping = false + + private val blockManager = SparkEnv.get.blockManager + private val shuffleBlockManager = blockManager.shuffleBlockManager + private val ser = Serializer.getSerializer(dep.serializer.getOrElse(null)) + private val shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser) + + /** Write a record to this task's output */ + override def write(record: Product2[K, V]): Unit = { + val pair = record.asInstanceOf[Product2[Any, Any]] + val bucketId = dep.partitioner.getPartition(pair._1) + shuffle.writers(bucketId).write(pair) + } + + /** Close this writer, passing along whether the map completed */ + override def stop(success: Boolean): Option[MapStatus] = { + try { + if (stopping) { + return None + } + stopping = true + if (success) { + try { + return Some(commitWritesAndBuildStatus()) + } catch { + case e: Exception => + revertWrites() + throw e + } + } else { + revertWrites() + return None + } + } finally { + // Release the writers back to the shuffle block manager. + if (shuffle != null && shuffle.writers != null) { + try { + shuffle.releaseWriters(success) + } catch { + case e: Exception => logError("Failed to release shuffle writers", e) + } + } + } + } + + private def commitWritesAndBuildStatus(): MapStatus = { + // Commit the writes. Get the size of each bucket block (total block size). + var totalBytes = 0L + var totalTime = 0L + val compressedSizes = shuffle.writers.map { writer: BlockObjectWriter => + writer.commit() + writer.close() + val size = writer.fileSegment().length + totalBytes += size + totalTime += writer.timeWriting() + MapOutputTracker.compressSize(size) + } + + // Update shuffle metrics. + val shuffleMetrics = new ShuffleWriteMetrics + shuffleMetrics.shuffleBytesWritten = totalBytes + shuffleMetrics.shuffleWriteTime = totalTime + metrics.shuffleWriteMetrics = Some(shuffleMetrics) + + new MapStatus(blockManager.blockManagerId, compressedSizes) + } + + private def revertWrites(): Unit = { + if (shuffle != null && shuffle.writers != null) { + for (writer <- shuffle.writers) { + writer.revertPartialWrites() + writer.close() + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala index c8f397609a0b4..22fdf73e9d1f4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala @@ -29,9 +29,9 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea setInitThread() private def setInitThread() { - // Set current thread as init thread - waitForReady will not block this thread - // (in case there is non trivial initialization which ends up calling waitForReady as part of - // initialization itself) + /* Set current thread as init thread - waitForReady will not block this thread + * (in case there is non trivial initialization which ends up calling waitForReady + * as part of initialization itself) */ BlockInfo.blockInfoInitThreads.put(this, Thread.currentThread()) } @@ -42,7 +42,9 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea def waitForReady(): Boolean = { if (pending && initThread != Thread.currentThread()) { synchronized { - while (pending) this.wait() + while (pending) { + this.wait() + } } } !failed @@ -50,8 +52,8 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea /** Mark this BlockInfo as ready (i.e. block is finished writing) */ def markReady(sizeInBytes: Long) { - require (sizeInBytes >= 0, "sizeInBytes was negative: " + sizeInBytes) - assert (pending) + require(sizeInBytes >= 0, s"sizeInBytes was negative: $sizeInBytes") + assert(pending) size = sizeInBytes BlockInfo.blockInfoInitThreads.remove(this) synchronized { @@ -61,7 +63,7 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea /** Mark this BlockInfo as ready but failed */ def markFailure() { - assert (pending) + assert(pending) size = BlockInfo.BLOCK_FAILED BlockInfo.blockInfoInitThreads.remove(this) synchronized { @@ -71,9 +73,9 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea } private object BlockInfo { - // initThread is logically a BlockInfo field, but we store it here because - // it's only needed while this block is in the 'pending' state and we want - // to minimize BlockInfo's memory footprint. + /* initThread is logically a BlockInfo field, but we store it here because + * it's only needed while this block is in the 'pending' state and we want + * to minimize BlockInfo's memory footprint. */ private val blockInfoInitThreads = new ConcurrentHashMap[BlockInfo, Thread] private val BLOCK_PENDING: Long = -1L diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6534095811907..f52bc7075104b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -28,46 +28,48 @@ import scala.util.Random import akka.actor.{ActorSystem, Cancellable, Props} import sun.nio.ch.DirectBuffer -import org.apache.spark.{Logging, MapOutputTracker, SecurityManager, SparkConf, SparkEnv, SparkException} +import org.apache.spark._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ -private[spark] sealed trait Values - -private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends Values -private[spark] case class IteratorValues(iterator: Iterator[Any]) extends Values -private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values +private[spark] sealed trait BlockValues +private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues +private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues +private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends BlockValues private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, val master: BlockManagerMaster, - val defaultSerializer: Serializer, + defaultSerializer: Serializer, maxMemory: Long, - val _conf: SparkConf, + val conf: SparkConf, securityManager: SecurityManager, mapOutputTracker: MapOutputTracker) extends Logging { - def conf = _conf val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + val connectionManager = new ConnectionManager(0, conf, securityManager) + + implicit val futureExecContext = connectionManager.futureExecContext private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] + // Actual storage of where blocks are kept + private var tachyonInitialized = false private[storage] val memoryStore = new MemoryStore(this, maxMemory) private[storage] val diskStore = new DiskStore(this, diskBlockManager) - var tachyonInitialized = false private[storage] lazy val tachyonStore: TachyonStore = { val storeDir = conf.get("spark.tachyonStore.baseDir", "/tmp_spark_tachyon") val appFolderName = conf.get("spark.tachyonStore.folderName") - val tachyonStorePath = s"${storeDir}/${appFolderName}/${this.executorId}" + val tachyonStorePath = s"$storeDir/$appFolderName/${this.executorId}" val tachyonMaster = conf.get("spark.tachyonStore.url", "tachyon://localhost:19998") - val tachyonBlockManager = new TachyonBlockManager( - shuffleBlockManager, tachyonStorePath, tachyonMaster) + val tachyonBlockManager = + new TachyonBlockManager(shuffleBlockManager, tachyonStorePath, tachyonMaster) tachyonInitialized = true new TachyonStore(this, tachyonBlockManager) } @@ -79,43 +81,39 @@ private[spark] class BlockManager( if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } - val connectionManager = new ConnectionManager(0, conf, securityManager) - implicit val futureExecContext = connectionManager.futureExecContext - val blockManagerId = BlockManagerId( executorId, connectionManager.id.host, connectionManager.id.port, nettyPort) // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) - val maxBytesInFlight = - conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024 + val maxBytesInFlight = conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) + private val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) // Whether to compress shuffle output that are stored - val compressShuffle = conf.getBoolean("spark.shuffle.compress", true) + private val compressShuffle = conf.getBoolean("spark.shuffle.compress", true) // Whether to compress RDD partitions that are stored serialized - val compressRdds = conf.getBoolean("spark.rdd.compress", false) + private val compressRdds = conf.getBoolean("spark.rdd.compress", false) // Whether to compress shuffle output temporarily spilled to disk - val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) - - val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) + private val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) - val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this, mapOutputTracker)), + private val slaveActor = actorSystem.actorOf( + Props(new BlockManagerSlaveActor(this, mapOutputTracker)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) - // Pending re-registration action being executed asynchronously or null if none - // is pending. Accesses should synchronize on asyncReregisterLock. - var asyncReregisterTask: Future[Unit] = null - val asyncReregisterLock = new Object + // Pending re-registration action being executed asynchronously or null if none is pending. + // Accesses should synchronize on asyncReregisterLock. + private var asyncReregisterTask: Future[Unit] = null + private val asyncReregisterLock = new Object - private def heartBeat() { + private def heartBeat(): Unit = { if (!master.sendHeartBeat(blockManagerId)) { reregister() } } - var heartBeatTask: Cancellable = null + private val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) + private var heartBeatTask: Cancellable = null private val metadataCleaner = new MetadataCleaner( MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf) @@ -124,11 +122,11 @@ private[spark] class BlockManager( initialize() - // The compression codec to use. Note that the "lazy" val is necessary because we want to delay - // the initialization of the compression codec until it is first used. The reason is that a Spark - // program could be using a user-defined codec in a third party jar, which is loaded in - // Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been - // loaded yet. + /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay + * the initialization of the compression codec until it is first used. The reason is that a Spark + * program could be using a user-defined codec in a third party jar, which is loaded in + * Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been + * loaded yet. */ private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) /** @@ -150,7 +148,7 @@ private[spark] class BlockManager( * Initialize the BlockManager. Register to the BlockManagerMaster, and start the * BlockManagerWorker actor. */ - private def initialize() { + private def initialize(): Unit = { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) BlockManagerWorker.startBlockManagerWorker(this) if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { @@ -170,12 +168,12 @@ private[spark] class BlockManager( * heart beat attempt or new block registration and another try to re-register all blocks * will be made then. */ - private def reportAllBlocks() { - logInfo("Reporting " + blockInfo.size + " blocks to the master.") + private def reportAllBlocks(): Unit = { + logInfo(s"Reporting ${blockInfo.size} blocks to the master.") for ((blockId, info) <- blockInfo) { val status = getCurrentBlockStatus(blockId, info) if (!tryToReportBlockStatus(blockId, info, status)) { - logError("Failed to report " + blockId + " to master; giving up.") + logError(s"Failed to report $blockId to master; giving up.") return } } @@ -187,7 +185,7 @@ private[spark] class BlockManager( * * Note that this method must be called without any BlockInfo locks held. */ - def reregister() { + private def reregister(): Unit = { // TODO: We might need to rate limit re-registering. logInfo("BlockManager re-registering with master") master.registerBlockManager(blockManagerId, maxMemory, slaveActor) @@ -197,7 +195,7 @@ private[spark] class BlockManager( /** * Re-register with the master sometime soon. */ - def asyncReregister() { + private def asyncReregister(): Unit = { asyncReregisterLock.synchronized { if (asyncReregisterTask == null) { asyncReregisterTask = Future[Unit] { @@ -213,7 +211,7 @@ private[spark] class BlockManager( /** * For testing. Wait for any pending asynchronous re-registration; otherwise, do nothing. */ - def waitForAsyncReregister() { + def waitForAsyncReregister(): Unit = { val task = asyncReregisterTask if (task != null) { Await.ready(task, Duration.Inf) @@ -251,18 +249,18 @@ private[spark] class BlockManager( * it is still valid). This ensures that update in master will compensate for the increase in * memory on slave. */ - def reportBlockStatus( + private def reportBlockStatus( blockId: BlockId, info: BlockInfo, status: BlockStatus, - droppedMemorySize: Long = 0L) { + droppedMemorySize: Long = 0L): Unit = { val needReregister = !tryToReportBlockStatus(blockId, info, status, droppedMemorySize) if (needReregister) { - logInfo("Got told to re-register updating block " + blockId) + logInfo(s"Got told to re-register updating block $blockId") // Re-registering will report our new block for free. asyncReregister() } - logDebug("Told master about block " + blockId) + logDebug(s"Told master about block $blockId") } /** @@ -293,10 +291,10 @@ private[spark] class BlockManager( * and the updated in-memory and on-disk sizes. */ private def getCurrentBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { - val (newLevel, inMemSize, onDiskSize, inTachyonSize) = info.synchronized { + info.synchronized { info.level match { case null => - (StorageLevel.NONE, 0L, 0L, 0L) + BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) case level => val inMem = level.useMemory && memoryStore.contains(blockId) val inTachyon = level.useOffHeap && tachyonStore.contains(blockId) @@ -307,19 +305,18 @@ private[spark] class BlockManager( val memSize = if (inMem) memoryStore.getSize(blockId) else 0L val tachyonSize = if (inTachyon) tachyonStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - (storageLevel, memSize, diskSize, tachyonSize) + BlockStatus(storageLevel, memSize, diskSize, tachyonSize) } } - BlockStatus(newLevel, inMemSize, onDiskSize, inTachyonSize) } /** * Get locations of an array of blocks. */ - def getLocationBlockIds(blockIds: Array[BlockId]): Array[Seq[BlockManagerId]] = { + private def getLocationBlockIds(blockIds: Array[BlockId]): Array[Seq[BlockManagerId]] = { val startTimeMs = System.currentTimeMillis val locations = master.getLocations(blockIds).toArray - logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs)) + logDebug("Got multiple block location in %s".format(Utils.getUsedTimeMs(startTimeMs))) locations } @@ -329,15 +326,16 @@ private[spark] class BlockManager( * never deletes (recent) items. */ def getLocalFromDisk(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { - diskStore.getValues(blockId, serializer).orElse( - sys.error("Block " + blockId + " not found on disk, though it should be")) + diskStore.getValues(blockId, serializer).orElse { + throw new BlockException(blockId, s"Block $blockId not found on disk, though it should be") + } } /** * Get block from local block manager. */ def getLocal(blockId: BlockId): Option[Iterator[Any]] = { - logDebug("Getting local block " + blockId) + logDebug(s"Getting local block $blockId") doGetLocal(blockId, asValues = true).asInstanceOf[Option[Iterator[Any]]] } @@ -345,7 +343,7 @@ private[spark] class BlockManager( * Get block from the local block manager as serialized bytes. */ def getLocalBytes(blockId: BlockId): Option[ByteBuffer] = { - logDebug("Getting local block " + blockId + " as bytes") + logDebug(s"Getting local block $blockId as bytes") // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work if (blockId.isShuffle) { @@ -353,7 +351,8 @@ private[spark] class BlockManager( case Some(bytes) => Some(bytes) case None => - throw new Exception("Block " + blockId + " not found on disk, though it should be") + throw new BlockException( + blockId, s"Block $blockId not found on disk, though it should be") } } else { doGetLocal(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] @@ -368,16 +367,16 @@ private[spark] class BlockManager( // If another thread is writing the block, wait for it to become ready. if (!info.waitForReady()) { // If we get here, the block write failed. - logWarning("Block " + blockId + " was marked as failure.") + logWarning(s"Block $blockId was marked as failure.") return None } val level = info.level - logDebug("Level for block " + blockId + " is " + level) + logDebug(s"Level for block $blockId is $level") // Look for the block in memory if (level.useMemory) { - logDebug("Getting block " + blockId + " from memory") + logDebug(s"Getting block $blockId from memory") val result = if (asValues) { memoryStore.getValues(blockId) } else { @@ -387,51 +386,51 @@ private[spark] class BlockManager( case Some(values) => return Some(values) case None => - logDebug("Block " + blockId + " not found in memory") + logDebug(s"Block $blockId not found in memory") } } // Look for the block in Tachyon if (level.useOffHeap) { - logDebug("Getting block " + blockId + " from tachyon") + logDebug(s"Getting block $blockId from tachyon") if (tachyonStore.contains(blockId)) { tachyonStore.getBytes(blockId) match { - case Some(bytes) => { + case Some(bytes) => if (!asValues) { return Some(bytes) } else { return Some(dataDeserialize(blockId, bytes)) } - } case None => - logDebug("Block " + blockId + " not found in tachyon") + logDebug(s"Block $blockId not found in tachyon") } } } - // Look for block on disk, potentially storing it back into memory if required: + // Look for block on disk, potentially storing it back in memory if required if (level.useDisk) { - logDebug("Getting block " + blockId + " from disk") + logDebug(s"Getting block $blockId from disk") val bytes: ByteBuffer = diskStore.getBytes(blockId) match { - case Some(bytes) => bytes + case Some(b) => b case None => - throw new Exception("Block " + blockId + " not found on disk, though it should be") + throw new BlockException( + blockId, s"Block $blockId not found on disk, though it should be") } - assert (0 == bytes.position()) + assert(0 == bytes.position()) if (!level.useMemory) { - // If the block shouldn't be stored in memory, we can just return it: + // If the block shouldn't be stored in memory, we can just return it if (asValues) { return Some(dataDeserialize(blockId, bytes)) } else { return Some(bytes) } } else { - // Otherwise, we also have to store something in the memory store: + // Otherwise, we also have to store something in the memory store if (!level.deserialized || !asValues) { - // We'll store the bytes in memory if the block's storage level includes - // "memory serialized", or if it should be cached as objects in memory - // but we only requested its serialized bytes: + /* We'll store the bytes in memory if the block's storage level includes + * "memory serialized", or if it should be cached as objects in memory + * but we only requested its serialized bytes. */ val copyForMemory = ByteBuffer.allocate(bytes.limit) copyForMemory.put(bytes) memoryStore.putBytes(blockId, copyForMemory, level) @@ -442,16 +441,17 @@ private[spark] class BlockManager( } else { val values = dataDeserialize(blockId, bytes) if (level.deserialized) { - // Cache the values before returning them: + // Cache the values before returning them // TODO: Consider creating a putValues that also takes in a iterator? val valuesBuffer = new ArrayBuffer[Any] valuesBuffer ++= values - memoryStore.putValues(blockId, valuesBuffer, level, true).data match { - case Left(values2) => - return Some(values2) - case _ => - throw new Exception("Memory store did not return back an iterator") - } + memoryStore.putValues(blockId, valuesBuffer, level, returnValues = true).data + match { + case Left(values2) => + return Some(values2) + case _ => + throw new SparkException("Memory store did not return an iterator") + } } else { return Some(values) } @@ -460,7 +460,7 @@ private[spark] class BlockManager( } } } else { - logDebug("Block " + blockId + " not registered locally") + logDebug(s"Block $blockId not registered locally") } None } @@ -469,7 +469,7 @@ private[spark] class BlockManager( * Get block from remote block managers. */ def getRemote(blockId: BlockId): Option[Iterator[Any]] = { - logDebug("Getting remote block " + blockId) + logDebug(s"Getting remote block $blockId") doGetRemote(blockId, asValues = true).asInstanceOf[Option[Iterator[Any]]] } @@ -477,7 +477,7 @@ private[spark] class BlockManager( * Get block from remote block managers as serialized bytes. */ def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { - logDebug("Getting remote block " + blockId + " as bytes") + logDebug(s"Getting remote block $blockId as bytes") doGetRemote(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] } @@ -485,7 +485,7 @@ private[spark] class BlockManager( require(blockId != null, "BlockId is null") val locations = Random.shuffle(master.getLocations(blockId)) for (loc <- locations) { - logDebug("Getting remote block " + blockId + " from " + loc) + logDebug(s"Getting remote block $blockId from $loc") val data = BlockManagerWorker.syncGetBlock( GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) if (data != null) { @@ -495,9 +495,9 @@ private[spark] class BlockManager( return Some(data) } } - logDebug("The value of block " + blockId + " is null") + logDebug(s"The value of block $blockId is null") } - logDebug("Block " + blockId + " not found") + logDebug(s"Block $blockId not found") None } @@ -507,12 +507,12 @@ private[spark] class BlockManager( def get(blockId: BlockId): Option[Iterator[Any]] = { val local = getLocal(blockId) if (local.isDefined) { - logInfo("Found block %s locally".format(blockId)) + logInfo(s"Found block $blockId locally") return local } val remote = getRemote(blockId) if (remote.isDefined) { - logInfo("Found block %s remotely".format(blockId)) + logInfo(s"Found block $blockId remotely") return remote } None @@ -533,7 +533,6 @@ private[spark] class BlockManager( } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) } - iter.initialize() iter } @@ -543,6 +542,7 @@ private[spark] class BlockManager( values: Iterator[Any], level: StorageLevel, tellMaster: Boolean): Seq[(BlockId, BlockStatus)] = { + require(values != null, "Values is null") doPut(blockId, IteratorValues(values), level, tellMaster) } @@ -562,8 +562,8 @@ private[spark] class BlockManager( } /** - * Put a new block of values to the block manager. Return a list of blocks updated as a - * result of this put. + * Put a new block of values to the block manager. + * Return a list of blocks updated as a result of this put. */ def put( blockId: BlockId, @@ -575,8 +575,8 @@ private[spark] class BlockManager( } /** - * Put a new block of serialized bytes to the block manager. Return a list of blocks updated - * as a result of this put. + * Put a new block of serialized bytes to the block manager. + * Return a list of blocks updated as a result of this put. */ def putBytes( blockId: BlockId, @@ -589,7 +589,7 @@ private[spark] class BlockManager( private def doPut( blockId: BlockId, - data: Values, + data: BlockValues, level: StorageLevel, tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { @@ -599,20 +599,18 @@ private[spark] class BlockManager( // Return value val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - // Remember the block's storage level so that we can correctly drop it to disk if it needs - // to be dropped right after it got put into memory. Note, however, that other threads will - // not be able to get() this block until we call markReady on its BlockInfo. + /* Remember the block's storage level so that we can correctly drop it to disk if it needs + * to be dropped right after it got put into memory. Note, however, that other threads will + * not be able to get() this block until we call markReady on its BlockInfo. */ val putBlockInfo = { val tinfo = new BlockInfo(level, tellMaster) // Do atomically ! val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) - if (oldBlockOpt.isDefined) { if (oldBlockOpt.get.waitForReady()) { - logWarning("Block " + blockId + " already exists on this machine; not re-adding it") + logWarning(s"Block $blockId already exists on this machine; not re-adding it") return updatedBlocks } - // TODO: So the block info exists - but previous attempt to load it (?) failed. // What do we do now ? Retry on it ? oldBlockOpt.get @@ -623,10 +621,10 @@ private[spark] class BlockManager( val startTimeMs = System.currentTimeMillis - // If we're storing values and we need to replicate the data, we'll want access to the values, - // but because our put will read the whole iterator, there will be no values left. For the - // case where the put serializes data, we'll remember the bytes, above; but for the case where - // it doesn't, such as deserialized storage, let's rely on the put returning an Iterator. + /* If we're storing values and we need to replicate the data, we'll want access to the values, + * but because our put will read the whole iterator, there will be no values left. For the + * case where the put serializes data, we'll remember the bytes, above; but for the case where + * it doesn't, such as deserialized storage, let's rely on the put returning an Iterator. */ var valuesAfterPut: Iterator[Any] = null // Ditto for the bytes after the put @@ -637,78 +635,62 @@ private[spark] class BlockManager( // If we're storing bytes, then initiate the replication before storing them locally. // This is faster as data is already serialized and ready to send. - val replicationFuture = if (data.isInstanceOf[ByteBufferValues] && level.replication > 1) { - // Duplicate doesn't copy the bytes, just creates a wrapper - val bufferView = data.asInstanceOf[ByteBufferValues].buffer.duplicate() - Future { - replicate(blockId, bufferView, level) - } - } else { - null + val replicationFuture = data match { + case b: ByteBufferValues if level.replication > 1 => + // Duplicate doesn't copy the bytes, but just creates a wrapper + val bufferView = b.buffer.duplicate() + Future { replicate(blockId, bufferView, level) } + case _ => null } putBlockInfo.synchronized { - logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) - + " to get into synchronized block") + logTrace("Put for block %s took %s to get into synchronized block" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) var marked = false try { - if (level.useMemory) { - // Save it just to memory first, even if it also has useDisk set to true; we will - // drop it to disk later if the memory store can't hold it. - val res = data match { - case IteratorValues(iterator) => - memoryStore.putValues(blockId, iterator, level, true) - case ArrayBufferValues(array) => - memoryStore.putValues(blockId, array, level, true) - case ByteBufferValues(bytes) => - bytes.rewind() - memoryStore.putBytes(blockId, bytes, level) - } - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case Left(newIterator) => valuesAfterPut = newIterator - } - // Keep track of which blocks are dropped from memory - res.droppedBlocks.foreach { block => updatedBlocks += block } - } else if (level.useOffHeap) { - // Save to Tachyon. - val res = data match { - case IteratorValues(iterator) => - tachyonStore.putValues(blockId, iterator, level, false) - case ArrayBufferValues(array) => - tachyonStore.putValues(blockId, array, level, false) - case ByteBufferValues(bytes) => - bytes.rewind() - tachyonStore.putBytes(blockId, bytes, level) - } - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case _ => - } - } else { - // Save directly to disk. - // Don't get back the bytes unless we replicate them. - val askForBytes = level.replication > 1 - - val res = data match { - case IteratorValues(iterator) => - diskStore.putValues(blockId, iterator, level, askForBytes) - case ArrayBufferValues(array) => - diskStore.putValues(blockId, array, level, askForBytes) - case ByteBufferValues(bytes) => - bytes.rewind() - diskStore.putBytes(blockId, bytes, level) - } - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case _ => + // returnValues - Whether to return the values put + // blockStore - The type of storage to put these values into + val (returnValues, blockStore: BlockStore) = { + if (level.useMemory) { + // Put it in memory first, even if it also has useDisk set to true; + // We will drop it to disk later if the memory store can't hold it. + (true, memoryStore) + } else if (level.useOffHeap) { + // Use tachyon for off-heap storage + (false, tachyonStore) + } else if (level.useDisk) { + // Don't get back the bytes from put unless we replicate them + (level.replication > 1, diskStore) + } else { + assert(level == StorageLevel.NONE) + throw new BlockException( + blockId, s"Attempted to put block $blockId without specifying storage level!") } } + // Actually put the values + val result = data match { + case IteratorValues(iterator) => + blockStore.putValues(blockId, iterator, level, returnValues) + case ArrayBufferValues(array) => + blockStore.putValues(blockId, array, level, returnValues) + case ByteBufferValues(bytes) => + bytes.rewind() + blockStore.putBytes(blockId, bytes, level) + } + size = result.size + result.data match { + case Left (newIterator) if level.useMemory => valuesAfterPut = newIterator + case Right (newBytes) => bytesAfterPut = newBytes + case _ => + } + + // Keep track of which blocks are dropped from memory + if (level.useMemory) { + result.droppedBlocks.foreach { updatedBlocks += _ } + } + val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) if (putBlockStatus.storageLevel != StorageLevel.NONE) { // Now that the block is in either the memory, tachyon, or disk store, @@ -728,18 +710,21 @@ private[spark] class BlockManager( // could've inserted a new BlockInfo before we remove it. blockInfo.remove(blockId) putBlockInfo.markFailure() - logWarning("Putting block " + blockId + " failed") + logWarning(s"Putting block $blockId failed") } } } - logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) + logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) // Either we're storing bytes and we asynchronously started replication, or we're storing // values and need to serialize and replicate them now: if (level.replication > 1) { data match { - case ByteBufferValues(bytes) => Await.ready(replicationFuture, Duration.Inf) - case _ => { + case ByteBufferValues(bytes) => + if (replicationFuture != null) { + Await.ready(replicationFuture, Duration.Inf) + } + case _ => val remoteStartTime = System.currentTimeMillis // Serialize the block if not already done if (bytesAfterPut == null) { @@ -750,20 +735,19 @@ private[spark] class BlockManager( bytesAfterPut = dataSerialize(blockId, valuesAfterPut) } replicate(blockId, bytesAfterPut, level) - logDebug("Put block " + blockId + " remotely took " + - Utils.getUsedTimeMs(remoteStartTime)) - } + logDebug("Put block %s remotely took %s" + .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } } BlockManager.dispose(bytesAfterPut) if (level.replication > 1) { - logDebug("Put for block " + blockId + " with replication took " + - Utils.getUsedTimeMs(startTimeMs)) + logDebug("Putting block %s with replication took %s" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { - logDebug("Put for block " + blockId + " without replication took " + - Utils.getUsedTimeMs(startTimeMs)) + logDebug("Putting block %s without replication took %s" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } updatedBlocks @@ -772,8 +756,8 @@ private[spark] class BlockManager( /** * Replicate block to another node. */ - var cachedPeers: Seq[BlockManagerId] = null - private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel) { + @volatile var cachedPeers: Seq[BlockManagerId] = null + private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel): Unit = { val tLevel = StorageLevel( level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) if (cachedPeers == null) { @@ -782,15 +766,16 @@ private[spark] class BlockManager( for (peer: BlockManagerId <- cachedPeers) { val start = System.nanoTime data.rewind() - logDebug("Try to replicate BlockId " + blockId + " once; The size of the data is " - + data.limit() + " Bytes. To node: " + peer) - if (!BlockManagerWorker.syncPutBlock(PutBlock(blockId, data, tLevel), - new ConnectionManagerId(peer.host, peer.port))) { - logError("Failed to call syncPutBlock to " + peer) + logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} Bytes. " + + s"To node: $peer") + val putBlock = PutBlock(blockId, data, tLevel) + val cmId = new ConnectionManagerId(peer.host, peer.port) + val syncPutBlockSuccess = BlockManagerWorker.syncPutBlock(putBlock, cmId) + if (!syncPutBlockSuccess) { + logError(s"Failed to call syncPutBlock to $peer") } - logDebug("Replicated BlockId " + blockId + " once used " + - (System.nanoTime - start) / 1e6 + " s; The size of the data is " + - data.limit() + " bytes.") + logDebug("Replicating BlockId %s once used %fs; The size of the data is %d bytes." + .format(blockId, (System.nanoTime - start) / 1e6, data.limit())) } } @@ -822,17 +807,17 @@ private[spark] class BlockManager( blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer]): Option[BlockStatus] = { - logInfo("Dropping block " + blockId + " from memory") + logInfo(s"Dropping block $blockId from memory") val info = blockInfo.get(blockId).orNull // If the block has not already been dropped - if (info != null) { + if (info != null) { info.synchronized { // required ? As of now, this will be invoked only for blocks which are ready // But in case this changes in future, adding for consistency sake. if (!info.waitForReady()) { // If we get here, the block write failed. - logWarning("Block " + blockId + " was marked as failure. Nothing to drop") + logWarning(s"Block $blockId was marked as failure. Nothing to drop") return None } @@ -841,10 +826,10 @@ private[spark] class BlockManager( // Drop to disk, if storage level requires if (level.useDisk && !diskStore.contains(blockId)) { - logInfo("Writing block " + blockId + " to disk") + logInfo(s"Writing block $blockId to disk") data match { case Left(elements) => - diskStore.putValues(blockId, elements, level, false) + diskStore.putValues(blockId, elements, level, returnValues = false) case Right(bytes) => diskStore.putBytes(blockId, bytes, level) } @@ -858,7 +843,7 @@ private[spark] class BlockManager( if (blockIsRemoved) { blockIsUpdated = true } else { - logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") + logWarning(s"Block $blockId could not be dropped from memory as it does not exist") } val status = getCurrentBlockStatus(blockId, info) @@ -883,7 +868,7 @@ private[spark] class BlockManager( */ def removeRdd(rddId: Int): Int = { // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. - logInfo("Removing RDD " + rddId) + logInfo(s"Removing RDD $rddId") val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) } blocksToRemove.size @@ -893,7 +878,7 @@ private[spark] class BlockManager( * Remove all blocks belonging to the given broadcast. */ def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { - logInfo("Removing broadcast " + broadcastId) + logInfo(s"Removing broadcast $broadcastId") val blocksToRemove = blockInfo.keys.collect { case bid @ BroadcastBlockId(`broadcastId`, _) => bid } @@ -904,40 +889,42 @@ private[spark] class BlockManager( /** * Remove a block from both memory and disk. */ - def removeBlock(blockId: BlockId, tellMaster: Boolean = true) { - logInfo("Removing block " + blockId) + def removeBlock(blockId: BlockId, tellMaster: Boolean = true): Unit = { + logInfo(s"Removing block $blockId") val info = blockInfo.get(blockId).orNull - if (info != null) info.synchronized { - // Removals are idempotent in disk store and memory store. At worst, we get a warning. - val removedFromMemory = memoryStore.remove(blockId) - val removedFromDisk = diskStore.remove(blockId) - val removedFromTachyon = if (tachyonInitialized) tachyonStore.remove(blockId) else false - if (!removedFromMemory && !removedFromDisk && !removedFromTachyon) { - logWarning("Block " + blockId + " could not be removed as it was not found in either " + - "the disk, memory, or tachyon store") - } - blockInfo.remove(blockId) - if (tellMaster && info.tellMaster) { - val status = getCurrentBlockStatus(blockId, info) - reportBlockStatus(blockId, info, status) + if (info != null) { + info.synchronized { + // Removals are idempotent in disk store and memory store. At worst, we get a warning. + val removedFromMemory = memoryStore.remove(blockId) + val removedFromDisk = diskStore.remove(blockId) + val removedFromTachyon = if (tachyonInitialized) tachyonStore.remove(blockId) else false + if (!removedFromMemory && !removedFromDisk && !removedFromTachyon) { + logWarning(s"Block $blockId could not be removed as it was not found in either " + + "the disk, memory, or tachyon store") + } + blockInfo.remove(blockId) + if (tellMaster && info.tellMaster) { + val status = getCurrentBlockStatus(blockId, info) + reportBlockStatus(blockId, info, status) + } } } else { // The block has already been removed; do nothing. - logWarning("Asked to remove block " + blockId + ", which does not exist") + logWarning(s"Asked to remove block $blockId, which does not exist") } } - private def dropOldNonBroadcastBlocks(cleanupTime: Long) { - logInfo("Dropping non broadcast blocks older than " + cleanupTime) + private def dropOldNonBroadcastBlocks(cleanupTime: Long): Unit = { + logInfo(s"Dropping non broadcast blocks older than $cleanupTime") dropOldBlocks(cleanupTime, !_.isBroadcast) } - private def dropOldBroadcastBlocks(cleanupTime: Long) { - logInfo("Dropping broadcast blocks older than " + cleanupTime) + private def dropOldBroadcastBlocks(cleanupTime: Long): Unit = { + logInfo(s"Dropping broadcast blocks older than $cleanupTime") dropOldBlocks(cleanupTime, _.isBroadcast) } - private def dropOldBlocks(cleanupTime: Long, shouldDrop: (BlockId => Boolean)) { + private def dropOldBlocks(cleanupTime: Long, shouldDrop: (BlockId => Boolean)): Unit = { val iterator = blockInfo.getEntrySet.iterator while (iterator.hasNext) { val entry = iterator.next() @@ -945,17 +932,11 @@ private[spark] class BlockManager( if (time < cleanupTime && shouldDrop(id)) { info.synchronized { val level = info.level - if (level.useMemory) { - memoryStore.remove(id) - } - if (level.useDisk) { - diskStore.remove(id) - } - if (level.useOffHeap) { - tachyonStore.remove(id) - } + if (level.useMemory) { memoryStore.remove(id) } + if (level.useDisk) { diskStore.remove(id) } + if (level.useOffHeap) { tachyonStore.remove(id) } iterator.remove() - logInfo("Dropped block " + id) + logInfo(s"Dropped block $id") } val status = getCurrentBlockStatus(id, info) reportBlockStatus(id, info, status) @@ -963,12 +944,14 @@ private[spark] class BlockManager( } } - def shouldCompress(blockId: BlockId): Boolean = blockId match { - case ShuffleBlockId(_, _, _) => compressShuffle - case BroadcastBlockId(_, _) => compressBroadcast - case RDDBlockId(_, _) => compressRdds - case TempBlockId(_) => compressShuffleSpill - case _ => false + private def shouldCompress(blockId: BlockId): Boolean = { + blockId match { + case _: ShuffleBlockId => compressShuffle + case _: BroadcastBlockId => compressBroadcast + case _: RDDBlockId => compressRdds + case _: TempBlockId => compressShuffleSpill + case _ => false + } } /** @@ -990,7 +973,7 @@ private[spark] class BlockManager( blockId: BlockId, outputStream: OutputStream, values: Iterator[Any], - serializer: Serializer = defaultSerializer) { + serializer: Serializer = defaultSerializer): Unit = { val byteStream = new BufferedOutputStream(outputStream) val ser = serializer.newInstance() ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() @@ -1015,11 +998,29 @@ private[spark] class BlockManager( bytes: ByteBuffer, serializer: Serializer = defaultSerializer): Iterator[Any] = { bytes.rewind() - val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true)) - serializer.newInstance().deserializeStream(stream).asIterator + + def getIterator: Iterator[Any] = { + val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true)) + serializer.newInstance().deserializeStream(stream).asIterator + } + + if (blockId.isShuffle) { + /* Reducer may need to read many local shuffle blocks and will wrap them into Iterators + * at the beginning. The wrapping will cost some memory (compression instance + * initialization, etc.). Reducer reads shuffle blocks one by one so we could do the + * wrapping lazily to save memory. */ + class LazyProxyIterator(f: => Iterator[Any]) extends Iterator[Any] { + lazy val proxy = f + override def hasNext: Boolean = proxy.hasNext + override def next(): Any = proxy.next() + } + new LazyProxyIterator(getIterator) + } else { + getIterator + } } - def stop() { + def stop(): Unit = { if (heartBeatTask != null) { heartBeatTask.cancel() } @@ -1041,9 +1042,9 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { - val ID_GENERATOR = new IdGenerator + private val ID_GENERATOR = new IdGenerator - def getMaxMemory(conf: SparkConf): Long = { + private def getMaxMemory(conf: SparkConf): Long = { val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6) (Runtime.getRuntime.maxMemory * memoryFraction).toLong } @@ -1060,9 +1061,9 @@ private[spark] object BlockManager extends Logging { * waiting for the GC to find it because that could lead to huge numbers of open files. There's * unfortunately no standard API to do this. */ - def dispose(buffer: ByteBuffer) { + def dispose(buffer: ByteBuffer): Unit = { if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) { - logTrace("Unmapping " + buffer) + logTrace(s"Unmapping $buffer") if (buffer.asInstanceOf[DirectBuffer].cleaner() != null) { buffer.asInstanceOf[DirectBuffer].cleaner().clean() } @@ -1075,7 +1076,7 @@ private[spark] object BlockManager extends Logging { blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[BlockManagerId]] = { // blockManagerMaster != null is used in tests - assert (env != null || blockManagerMaster != null) + assert(env != null || blockManagerMaster != null) val blockLocations: Seq[Seq[BlockManagerId]] = if (blockManagerMaster == null) { env.blockManager.getLocationBlockIds(blockIds) } else { diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 3a7243a1ba19c..2ec46d416f37d 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -40,9 +40,9 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 private val subDirsPerLocalDir = shuffleManager.conf.getInt("spark.diskStore.subDirectories", 64) - // Create one local directory for each path mentioned in spark.local.dir; then, inside this - // directory, create multiple subdirectories that we will hash files into, in order to avoid - // having really large inodes at the top level. + /* Create one local directory for each path mentioned in spark.local.dir; then, inside this + * directory, create multiple subdirectories that we will hash files into, in order to avoid + * having really large inodes at the top level. */ private val localDirs: Array[File] = createLocalDirs() private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) private var shuffleSender : ShuffleSender = null @@ -114,7 +114,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD } private def createLocalDirs(): Array[File] = { - logDebug("Creating local directories at root dirs '" + rootDirs + "'") + logDebug(s"Creating local directories at root dirs '$rootDirs'") val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") rootDirs.split(",").map { rootDir => var foundLocalDir = false @@ -126,21 +126,20 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD tries += 1 try { localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) - localDir = new File(rootDir, "spark-local-" + localDirId) + localDir = new File(rootDir, s"spark-local-$localDirId") if (!localDir.exists) { foundLocalDir = localDir.mkdirs() } } catch { case e: Exception => - logWarning("Attempt " + tries + " to create local dir " + localDir + " failed", e) + logWarning(s"Attempt $tries to create local dir $localDir failed", e) } } if (!foundLocalDir) { - logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + - " attempts to create local dir in " + rootDir) + logError(s"Failed $MAX_DIR_CREATION_ATTEMPTS attempts to create local dir in $rootDir") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } - logInfo("Created local directory at " + localDir) + logInfo(s"Created local directory at $localDir") localDir } } @@ -163,7 +162,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) } catch { case e: Exception => - logError("Exception while deleting local spark dir: " + localDir, e) + logError(s"Exception while deleting local spark dir: $localDir", e) } } } @@ -175,7 +174,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD private[storage] def startShuffleBlockSender(port: Int): Int = { shuffleSender = new ShuffleSender(port, this) - logInfo("Created ShuffleSender binding to port : " + shuffleSender.port) + logInfo(s"Created ShuffleSender binding to port: ${shuffleSender.port}") shuffleSender.port } } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 0ab9fad422717..ebff0cb5ba153 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -39,41 +39,39 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage diskManager.getBlockLocation(blockId).length } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) : PutResult = { + override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { // So that we do not modify the input offsets ! // duplicate does not copy buffer, so inexpensive val bytes = _bytes.duplicate() - logDebug("Attempting to put block " + blockId) + logDebug(s"Attempting to put block $blockId") val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) - val channel = new FileOutputStream(file).getChannel() + val channel = new FileOutputStream(file).getChannel while (bytes.remaining > 0) { channel.write(bytes) } channel.close() val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file on disk in %d ms".format( - file.getName, Utils.bytesToString(bytes.limit), (finishTime - startTime))) - return PutResult(bytes.limit(), Right(bytes.duplicate())) + file.getName, Utils.bytesToString(bytes.limit), finishTime - startTime)) + PutResult(bytes.limit(), Right(bytes.duplicate())) } override def putValues( blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - returnValues: Boolean) - : PutResult = { - return putValues(blockId, values.toIterator, level, returnValues) + returnValues: Boolean): PutResult = { + putValues(blockId, values.toIterator, level, returnValues) } override def putValues( blockId: BlockId, values: Iterator[Any], level: StorageLevel, - returnValues: Boolean) - : PutResult = { + returnValues: Boolean): PutResult = { - logDebug("Attempting to write values for block " + blockId) + logDebug(s"Attempting to write values for block $blockId") val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) val outputStream = new FileOutputStream(file) @@ -95,7 +93,7 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage override def getBytes(blockId: BlockId): Option[ByteBuffer] = { val segment = diskManager.getBlockLocation(blockId) - val channel = new RandomAccessFile(segment.file, "r").getChannel() + val channel = new RandomAccessFile(segment.file, "r").getChannel try { // For small files, directly read rather than memory map @@ -131,7 +129,7 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage file.delete() } else { if (fileSegment.length < file.length()) { - logWarning("Could not delete block associated with only a part of a file: " + blockId) + logWarning(s"Could not delete block associated with only a part of a file: $blockId") } false } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 488f1ea9628f5..084a566c48560 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -24,6 +24,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.util.{SizeEstimator, Utils} +private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) + /** * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as * serialized ByteBuffers. @@ -31,15 +33,13 @@ import org.apache.spark.util.{SizeEstimator, Utils} private class MemoryStore(blockManager: BlockManager, maxMemory: Long) extends BlockStore(blockManager) { - case class Entry(value: Any, size: Long, deserialized: Boolean) - - private val entries = new LinkedHashMap[BlockId, Entry](32, 0.75f, true) + private val entries = new LinkedHashMap[BlockId, MemoryEntry](32, 0.75f, true) @volatile private var currentMemory = 0L // Object used to ensure that only one thread is putting blocks and if necessary, dropping // blocks from the memory store. private val putLock = new Object() - logInfo("MemoryStore started with capacity %s.".format(Utils.bytesToString(maxMemory))) + logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) def freeMemory: Long = maxMemory - currentMemory @@ -101,7 +101,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else if (entry.deserialized) { Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[ArrayBuffer[Any]].iterator)) } else { - Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data + Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data } } @@ -124,8 +124,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) val entry = entries.remove(blockId) if (entry != null) { currentMemory -= entry.size - logInfo("Block %s of size %d dropped from memory (free %d)".format( - blockId, entry.size, freeMemory)) + logInfo(s"Block $blockId of size ${entry.size} dropped from memory (free $freeMemory)") true } else { false @@ -181,18 +180,14 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) droppedBlocks ++= freeSpaceResult.droppedBlocks if (enoughFreeSpace) { - val entry = new Entry(value, size, deserialized) + val entry = new MemoryEntry(value, size, deserialized) entries.synchronized { entries.put(blockId, entry) currentMemory += size } - if (deserialized) { - logInfo("Block %s stored as values to memory (estimated size %s, free %s)".format( - blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) - } else { - logInfo("Block %s stored as bytes to memory (size %s, free %s)".format( - blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) - } + val valuesOrBytes = if (deserialized) "values" else "bytes" + logInfo("Block %s stored as %s in memory (estimated size %s, free %s)".format( + blockId, valuesOrBytes, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) putSuccess = true } else { // Tell the block manager that we couldn't put it in memory so that it can drop it to @@ -221,13 +216,12 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * Return whether there is enough free space, along with the blocks dropped in the process. */ private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): ResultWithDroppedBlocks = { - logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format( - space, currentMemory, maxMemory)) + logInfo(s"ensureFreeSpace($space) called with curMem=$currentMemory, maxMem=$maxMemory") val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] if (space > maxMemory) { - logInfo("Will not store " + blockIdToAdd + " as it is larger than our memory limit") + logInfo(s"Will not store $blockIdToAdd as it is larger than our memory limit") return ResultWithDroppedBlocks(success = false, droppedBlocks) } @@ -252,7 +246,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } if (maxMemory - (currentMemory - selectedMemory) >= space) { - logInfo(selectedBlocks.size + " blocks selected for dropping") + logInfo(s"${selectedBlocks.size} blocks selected for dropping") for (blockId <- selectedBlocks) { val entry = entries.synchronized { entries.get(blockId) } // This should never be null as only one thread should be dropping diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 023fd6e4d8baa..5a72e216872a6 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -26,7 +26,7 @@ class RDDInfo( val id: Int, val name: String, val numPartitions: Int, - val storageLevel: StorageLevel) + var storageLevel: StorageLevel) extends Ordered[RDDInfo] { var numCachedPartitions = 0 @@ -36,8 +36,8 @@ class RDDInfo( override def toString = { import Utils.bytesToString - ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + - "TachyonSize: %s; DiskSize: %s").format( + ("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " + + "MemorySize: %s; TachyonSize: %s; DiskSize: %s").format( name, id, storageLevel.toString, numCachedPartitions, numPartitions, bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 363de93e067b8..1e35abaab5353 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -34,11 +34,11 @@ import org.apache.spark.annotation.DeveloperApi */ @DeveloperApi class StorageLevel private( - private var useDisk_ : Boolean, - private var useMemory_ : Boolean, - private var useOffHeap_ : Boolean, - private var deserialized_ : Boolean, - private var replication_ : Int = 1) + private var _useDisk: Boolean, + private var _useMemory: Boolean, + private var _useOffHeap: Boolean, + private var _deserialized: Boolean, + private var _replication: Int = 1) extends Externalizable { // TODO: Also add fields for caching priority, dataset ID, and flushing. @@ -48,13 +48,13 @@ class StorageLevel private( def this() = this(false, true, false, false) // For deserialization - def useDisk = useDisk_ - def useMemory = useMemory_ - def useOffHeap = useOffHeap_ - def deserialized = deserialized_ - def replication = replication_ + def useDisk = _useDisk + def useMemory = _useMemory + def useOffHeap = _useOffHeap + def deserialized = _deserialized + def replication = _replication - assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") + assert(replication < 40, "Replication restricted to be less than 40 for calculating hash codes") if (useOffHeap) { require(!useDisk, "Off-heap storage level does not support using disk") @@ -63,8 +63,9 @@ class StorageLevel private( require(replication == 1, "Off-heap storage level does not support multiple replication") } - override def clone(): StorageLevel = new StorageLevel( - this.useDisk, this.useMemory, this.useOffHeap, this.deserialized, this.replication) + override def clone(): StorageLevel = { + new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication) + } override def equals(other: Any): Boolean = other match { case s: StorageLevel => @@ -77,20 +78,20 @@ class StorageLevel private( false } - def isValid = ((useMemory || useDisk || useOffHeap) && (replication > 0)) + def isValid = (useMemory || useDisk || useOffHeap) && (replication > 0) def toInt: Int = { var ret = 0 - if (useDisk_) { + if (_useDisk) { ret |= 8 } - if (useMemory_) { + if (_useMemory) { ret |= 4 } - if (useOffHeap_) { + if (_useOffHeap) { ret |= 2 } - if (deserialized_) { + if (_deserialized) { ret |= 1 } ret @@ -98,32 +99,34 @@ class StorageLevel private( override def writeExternal(out: ObjectOutput) { out.writeByte(toInt) - out.writeByte(replication_) + out.writeByte(_replication) } override def readExternal(in: ObjectInput) { val flags = in.readByte() - useDisk_ = (flags & 8) != 0 - useMemory_ = (flags & 4) != 0 - useOffHeap_ = (flags & 2) != 0 - deserialized_ = (flags & 1) != 0 - replication_ = in.readByte() + _useDisk = (flags & 8) != 0 + _useMemory = (flags & 4) != 0 + _useOffHeap = (flags & 2) != 0 + _deserialized = (flags & 1) != 0 + _replication = in.readByte() } @throws(classOf[IOException]) private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) - override def toString: String = "StorageLevel(%b, %b, %b, %b, %d)".format( - useDisk, useMemory, useOffHeap, deserialized, replication) + override def toString: String = { + s"StorageLevel($useDisk, $useMemory, $useOffHeap, $deserialized, $replication)" + } override def hashCode(): Int = toInt * 41 + replication - def description : String = { + + def description: String = { var result = "" result += (if (useDisk) "Disk " else "") result += (if (useMemory) "Memory " else "") result += (if (useOffHeap) "Tachyon " else "") result += (if (deserialized) "Deserialized " else "Serialized ") - result += "%sx Replicated".format(replication) + result += s"${replication}x Replicated" result } } @@ -147,31 +150,63 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) + /** + * :: DeveloperApi :: + * Return the StorageLevel object with the specified name. + */ + @DeveloperApi + def fromString(s: String): StorageLevel = s match { + case "NONE" => NONE + case "DISK_ONLY" => DISK_ONLY + case "DISK_ONLY_2" => DISK_ONLY_2 + case "MEMORY_ONLY" => MEMORY_ONLY + case "MEMORY_ONLY_2" => MEMORY_ONLY_2 + case "MEMORY_ONLY_SER" => MEMORY_ONLY_SER + case "MEMORY_ONLY_SER_2" => MEMORY_ONLY_SER_2 + case "MEMORY_AND_DISK" => MEMORY_AND_DISK + case "MEMORY_AND_DISK_2" => MEMORY_AND_DISK_2 + case "MEMORY_AND_DISK_SER" => MEMORY_AND_DISK_SER + case "MEMORY_AND_DISK_SER_2" => MEMORY_AND_DISK_SER_2 + case "OFF_HEAP" => OFF_HEAP + case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $s") + } + /** * :: DeveloperApi :: * Create a new StorageLevel object without setting useOffHeap. */ @DeveloperApi - def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, - deserialized: Boolean, replication: Int) = getCachedStorageLevel( + def apply( + useDisk: Boolean, + useMemory: Boolean, + useOffHeap: Boolean, + deserialized: Boolean, + replication: Int) = { + getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) + } /** * :: DeveloperApi :: * Create a new StorageLevel object. */ @DeveloperApi - def apply(useDisk: Boolean, useMemory: Boolean, - deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( - new StorageLevel(useDisk, useMemory, false, deserialized, replication)) + def apply( + useDisk: Boolean, + useMemory: Boolean, + deserialized: Boolean, + replication: Int = 1) = { + getCachedStorageLevel(new StorageLevel(useDisk, useMemory, false, deserialized, replication)) + } /** * :: DeveloperApi :: * Create a new StorageLevel object from its integer representation. */ @DeveloperApi - def apply(flags: Int, replication: Int): StorageLevel = + def apply(flags: Int, replication: Int): StorageLevel = { getCachedStorageLevel(new StorageLevel(flags, replication)) + } /** * :: DeveloperApi :: @@ -184,8 +219,8 @@ object StorageLevel { getCachedStorageLevel(obj) } - private[spark] - val storageLevelCache = new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() + private[spark] val storageLevelCache = + new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() private[spark] def getCachedStorageLevel(level: StorageLevel): StorageLevel = { storageLevelCache.putIfAbsent(level, level) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index a6e6627d54e01..c694fc8c347ec 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -37,7 +37,11 @@ class StorageStatusListener extends SparkListener { val filteredStatus = storageStatusList.find(_.blockManagerId.executorId == execId) filteredStatus.foreach { storageStatus => updatedBlocks.foreach { case (blockId, updatedStatus) => - storageStatus.blocks(blockId) = updatedStatus + if (updatedStatus.storageLevel == StorageLevel.NONE) { + storageStatus.blocks.remove(blockId) + } else { + storageStatus.blocks(blockId) = updatedStatus + } } } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 6f3252a2f6d31..f3bde1df45c79 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -89,10 +89,13 @@ private[spark] object StorageUtils { // Add up memory, disk and Tachyon sizes val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize + status.tachyonSize > 0 } + val _storageLevel = + if (persistedBlocks.length > 0) persistedBlocks(0).storageLevel else StorageLevel.NONE val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) val tachyonSize = persistedBlocks.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) rddInfoMap.get(rddId).map { rddInfo => + rddInfo.storageLevel = _storageLevel rddInfo.numCachedPartitions = persistedBlocks.length rddInfo.memSize = memSize rddInfo.diskSize = diskSize diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index c37e76f893605..d8ff4ff6bd42c 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -22,15 +22,10 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import tachyon.client.{WriteType, ReadType} +import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging import org.apache.spark.util.Utils -import org.apache.spark.serializer.Serializer - - -private class Entry(val size: Long) - /** * Stores BlockManager blocks on Tachyon. @@ -46,8 +41,8 @@ private class TachyonStore( tachyonManager.getFile(blockId.name).length } - override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { - putToTachyonStore(blockId, bytes, true) + override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { + putIntoTachyonStore(blockId, bytes, returnValues = true) } override def putValues( @@ -55,7 +50,7 @@ private class TachyonStore( values: ArrayBuffer[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - return putValues(blockId, values.toIterator, level, returnValues) + putValues(blockId, values.toIterator, level, returnValues) } override def putValues( @@ -63,12 +58,12 @@ private class TachyonStore( values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - logDebug("Attempting to write values for block " + blockId) - val _bytes = blockManager.dataSerialize(blockId, values) - putToTachyonStore(blockId, _bytes, returnValues) + logDebug(s"Attempting to write values for block $blockId") + val bytes = blockManager.dataSerialize(blockId, values) + putIntoTachyonStore(blockId, bytes, returnValues) } - private def putToTachyonStore( + private def putIntoTachyonStore( blockId: BlockId, bytes: ByteBuffer, returnValues: Boolean): PutResult = { @@ -76,7 +71,7 @@ private class TachyonStore( // duplicate does not copy buffer, so inexpensive val byteBuffer = bytes.duplicate() byteBuffer.rewind() - logDebug("Attempting to put block " + blockId + " into Tachyon") + logDebug(s"Attempting to put block $blockId into Tachyon") val startTime = System.currentTimeMillis val file = tachyonManager.getFile(blockId) val os = file.getOutStream(WriteType.TRY_CACHE) @@ -84,7 +79,7 @@ private class TachyonStore( os.close() val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file in Tachyon in %d ms".format( - blockId, Utils.bytesToString(byteBuffer.limit), (finishTime - startTime))) + blockId, Utils.bytesToString(byteBuffer.limit), finishTime - startTime)) if (returnValues) { PutResult(bytes.limit(), Right(bytes.duplicate())) @@ -106,10 +101,9 @@ private class TachyonStore( getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) } - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { val file = tachyonManager.getFile(blockId) - if (file == null || file.getLocationHosts().size == 0) { + if (file == null || file.getLocationHosts.size == 0) { return None } val is = file.getInStream(ReadType.CACHE) @@ -121,16 +115,15 @@ private class TachyonStore( val fetchSize = is.read(bs, 0, size.asInstanceOf[Int]) buffer = ByteBuffer.wrap(bs) if (fetchSize != size) { - logWarning("Failed to fetch the block " + blockId + " from Tachyon : Size " + size + - " is not equal to fetched size " + fetchSize) + logWarning(s"Failed to fetch the block $blockId from Tachyon: Size $size " + + s"is not equal to fetched size $fetchSize") return None } } } catch { - case ioe: IOException => { - logWarning("Failed to fetch the block " + blockId + " from Tachyon", ioe) - return None - } + case ioe: IOException => + logWarning(s"Failed to fetch the block $blockId from Tachyon", ioe) + return None } Some(buffer) } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a43314f48112f..1b104253d545d 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -168,7 +168,7 @@ private[spark] object UIUtils extends Logging {